From c7b5e6a1634d66af5f6fb9344861e4ac22e5fdaa Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 25 May 2023 11:54:49 +0800 Subject: [PATCH] Revert "feat(backfill): Persist backfill operator state (#9752)" This reverts commit 5e80fd7f8f4907b4f7e2ba9fe08809eb34cd00d5. --- ci/scripts/gen-flamegraph.sh | 1 - proto/stream_plan.proto | 3 - src/common/src/hash/consistent_hash/bitmap.rs | 5 - src/common/src/hash/consistent_hash/vnode.rs | 9 +- src/common/src/util/stream_graph_visitor.rs | 5 - .../testdata/output/distribution_derive.yaml | 183 +---- .../testdata/output/emit_on_window_close.yaml | 10 +- .../tests/testdata/output/except.yaml | 40 +- .../tests/testdata/output/intersect.yaml | 40 +- .../tests/testdata/output/join.yaml | 12 +- .../tests/testdata/output/nexmark.yaml | 227 ++----- .../testdata/output/over_window_function.yaml | 10 +- .../tests/testdata/output/share.yaml | 9 - .../testdata/output/stream_dist_agg.yaml | 309 +-------- .../testdata/output/temporal_filter.yaml | 15 +- .../tests/testdata/output/tpch.yaml | 637 ++++++------------ .../tests/testdata/output/union.yaml | 44 +- src/frontend/src/optimizer/plan_node/mod.rs | 2 +- .../optimizer/plan_node/stream_table_scan.rs | 63 +- .../src/optimizer/property/distribution.rs | 2 +- .../src/utils/stream_graph_formatter.rs | 6 - src/meta/src/manager/catalog/mod.rs | 108 +-- src/meta/src/rpc/ddl_controller.rs | 25 +- src/meta/src/stream/stream_manager.rs | 2 +- src/stream/src/executor/backfill.rs | 521 +++++--------- src/stream/src/from_proto/chain.rs | 21 +- .../tests/integration_tests/main.rs | 1 - .../integration_tests/recovery/backfill.rs | 206 ------ .../tests/integration_tests/recovery/mod.rs | 1 - 29 files changed, 490 insertions(+), 2027 deletions(-) delete mode 100644 src/tests/simulation/tests/integration_tests/recovery/backfill.rs diff --git a/ci/scripts/gen-flamegraph.sh b/ci/scripts/gen-flamegraph.sh index 13137d88502a2..5286873de7ce7 100755 --- a/ci/scripts/gen-flamegraph.sh +++ b/ci/scripts/gen-flamegraph.sh @@ -77,7 +77,6 @@ install_all() { # faster addr2line to speed up heap flamegraph analysis by jeprof echo ">>> Installing addr2line" git clone https://github.com/gimli-rs/addr2line - cp risingwave/rust-toolchain addr2line/rust-toolchain pushd addr2line cargo b --examples -r mv ./target/release/examples/addr2line $(which addr2line) diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index a3fd013c86f0d..99485c34524e4 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -419,9 +419,6 @@ message ChainNode { // ChainType is used to decide which implementation for the ChainNode. ChainType chain_type = 4; - /// The state table used by Backfill operator for persisting internal state - catalog.Table state_table = 5; - // The upstream materialized view info used by backfill. plan_common.StorageTableDesc table_desc = 7; } diff --git a/src/common/src/hash/consistent_hash/bitmap.rs b/src/common/src/hash/consistent_hash/bitmap.rs index 19e9770fb4a98..c007138feb76f 100644 --- a/src/common/src/hash/consistent_hash/bitmap.rs +++ b/src/common/src/hash/consistent_hash/bitmap.rs @@ -25,11 +25,6 @@ impl Bitmap { self.iter_ones().map(VirtualNode::from_index) } - /// Enumerates the virtual nodes set to 1 in the bitmap. - pub fn iter_vnodes_scalar(&self) -> impl Iterator + '_ { - self.iter_vnodes().map(|vnode| vnode.to_scalar()) - } - /// Returns an iterator which yields the position ranges of continuous virtual nodes set to 1 in /// the bitmap. pub fn vnode_ranges(&self) -> impl Iterator> + '_ { diff --git a/src/common/src/hash/consistent_hash/vnode.rs b/src/common/src/hash/consistent_hash/vnode.rs index 0ee3596c82d8e..df5c49fbd78ad 100644 --- a/src/common/src/hash/consistent_hash/vnode.rs +++ b/src/common/src/hash/consistent_hash/vnode.rs @@ -18,7 +18,7 @@ use parse_display::Display; use crate::array::{Array, ArrayImpl, DataChunk}; use crate::hash::Crc32HashCode; use crate::row::{Row, RowExt}; -use crate::types::{DataType, ScalarRefImpl}; +use crate::types::ScalarRefImpl; use crate::util::hash_util::Crc32FastBuilder; use crate::util::row_id::extract_vnode_id_from_row_id; @@ -64,9 +64,6 @@ pub type AllVirtualNodeIter = std::iter::Map, fn(usize) - impl VirtualNode { /// The maximum value of the virtual node. pub const MAX: VirtualNode = VirtualNode::from_index(Self::COUNT - 1); - /// We may use `VirtualNode` as a datum in a stream, or store it as a column. - /// Hence this reifies it as a RW datatype. - pub const RW_TYPE: DataType = DataType::Int16; /// The minimum (zero) value of the virtual node. pub const ZERO: VirtualNode = VirtualNode::from_index(0); @@ -132,8 +129,8 @@ impl VirtualNode { .collect() } - // `compute_row` is used to calculate the `VirtualNode` for the corresponding columns in a - // `Row`. Similar to `compute_chunk`, it also contains special handling for serial columns. + // `compute_row` is used to calculate the `VirtualNode` for the corresponding column in a `Row`. + // Similar to `compute_chunk`, it also contains special handling for serial columns. pub fn compute_row(row: impl Row, indices: &[usize]) -> VirtualNode { let project = row.project(indices); if let Ok(Some(ScalarRefImpl::Serial(s))) = project.iter().exactly_one().as_ref() { diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index 6f72a8fca803e..1d45ec05616ca 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -178,11 +178,6 @@ fn visit_stream_node_tables_inner( always!(node.state_table, "Sort"); } - // Chain - NodeBody::Chain(node) => { - always!(node.state_table, "Chain") - } - // Note: add internal tables for new nodes here. NodeBody::Materialize(node) if !internal_tables_only => { always!(node.table, "Materialize") 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 291b059df28aa..8f9a00d75e80b 100644 --- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml @@ -34,12 +34,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 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: 0 } + Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } ├── Upstream └── BatchPlanNode @@ -53,10 +53,6 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, ak1_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 1 { columns: [ vnode, k1, b__row_id, bk1_backfill_finished ], 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: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id ], primary key: [ $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 3 } - id: Ak1_join_B_onk1 @@ -86,12 +82,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 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: 0 } + Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } ├── Upstream └── BatchPlanNode @@ -105,10 +101,6 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, ak1_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 1 { columns: [ vnode, k1, b__row_id, bk1_backfill_finished ], 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: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id ], primary key: [ $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 3 } - id: A_join_Bk1_onk1 @@ -138,12 +130,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 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: 0 } + Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } ├── Upstream └── BatchPlanNode @@ -157,10 +149,6 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, ak1_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 1 { columns: [ vnode, k1, b__row_id, bk1_backfill_finished ], 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: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id ], primary key: [ $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 3 } - id: Ak1_join_Bk1_onk1 @@ -190,12 +178,12 @@ └── StreamExchange Hash([2, 4, 3]) from 5 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: 0 } + Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } { state table: 1 } + Chain { table: bk1, columns: [bk1.k1, bk1.v, bk1.b._row_id], pk: [bk1.b._row_id], dist: UpstreamHashShard(bk1.k1) } ├── Upstream └── BatchPlanNode @@ -209,10 +197,6 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, ak1_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 1 { columns: [ vnode, k1, b__row_id, bk1_backfill_finished ], 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: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id ], primary key: [ $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 3 } - id: aggk1_from_A @@ -247,7 +231,6 @@ Fragment 1 Chain { table: a, columns: [a.k1, a.v, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -265,14 +248,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, _row_id, a_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ max_v, a.k1 ] ├── primary key: [ $1 ASC ] @@ -307,7 +282,6 @@ ├── state tables: [ 0 ] ├── distinct tables: [] └── Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -325,14 +299,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, k1, a__row_id, ak1_backfill_finished ] - ├── 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: [ max_v, ak1.k1 ] ├── primary key: [ $1 ASC ] @@ -372,7 +338,6 @@ Fragment 1 Chain { table: ak1k2, columns: [ak1k2.k1, ak1k2.v, ak1k2.k2, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -390,14 +355,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, k1, k2, a__row_id, ak1k2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ max_v, ak1k2.k1 ] ├── primary key: [ $1 ASC ] @@ -437,7 +394,6 @@ Fragment 1 Chain { table: ak1k2, columns: [ak1k2.k2, ak1k2.v, ak1k2.k1, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -455,14 +411,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, k1, k2, a__row_id, ak1k2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ max_v, ak1k2.k2 ] ├── primary key: [ $1 ASC ] @@ -494,7 +442,6 @@ └── StreamProject { exprs: [sum(ak1k2.v), ak1k2.k1, ak1k2.k2] } └── StreamHashAgg { group_key: [ak1k2.k1, ak1k2.k2], aggs: [sum(ak1k2.v), count] } { result table: 0, state tables: [], distinct tables: [] } └── Chain { table: ak1k2, columns: [ak1k2.k1, ak1k2.k2, ak1k2.v, ak1k2.a._row_id], pk: [ak1k2.a._row_id], dist: UpstreamHashShard(ak1k2.k1, ak1k2.k2) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -505,14 +452,6 @@ ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 - ├── columns: [ vnode, k1, k2, a__row_id, ak1k2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ sum_v, ak1k2.k1, ak1k2.k2 ] ├── primary key: [ $1 ASC, $2 ASC ] @@ -544,7 +483,6 @@ └── StreamProject { exprs: [sum(ak1.v), ak1.k1, ak1.k2] } └── StreamHashAgg { group_key: [ak1.k1, ak1.k2], aggs: [sum(ak1.v), count] } { result table: 0, state tables: [], distinct tables: [] } └── Chain { table: ak1, columns: [ak1.k1, ak1.k2, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -555,14 +493,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 2 - Table 1 - ├── columns: [ vnode, k1, a__row_id, ak1_backfill_finished ] - ├── 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: [ sum_v, ak1.k1, ak1.k2 ] ├── primary key: [ $1 ASC, $2 ASC ] @@ -614,7 +544,6 @@ Fragment 1 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -639,14 +568,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 - ├── columns: [ vnode, _row_id, a_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ max_num, a.k1 ] ├── primary key: [ $1 ASC ] @@ -713,7 +634,6 @@ Fragment 2 Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -738,14 +658,6 @@ ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 3 - ├── columns: [ vnode, _row_id, a_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ max_num, a.k1 ] ├── primary key: [ $1 ASC ] @@ -812,7 +724,6 @@ Fragment 2 Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -837,14 +748,6 @@ ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 3 - ├── columns: [ vnode, _row_id, a_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ max_num, a.k2 ] ├── primary key: [ $1 ASC ] @@ -889,7 +792,7 @@ └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 3 } + Chain { table: a, columns: [a.k1, a.k2, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── Upstream └── BatchPlanNode @@ -914,14 +817,6 @@ ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 3 - ├── columns: [ vnode, _row_id, a_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ max_num, a.k1, a.k2 ] ├── primary key: [ $1 ASC, $2 ASC ] @@ -971,16 +866,16 @@ ├── 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] } { result table: 5, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { result table: 4, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 2 Fragment 1 - Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 4 } + Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 6 } + Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── Upstream └── BatchPlanNode @@ -992,11 +887,7 @@ Table 3 { columns: [ a_k1, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, k1, a__row_id, ak1_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 5 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 6 { columns: [ vnode, _row_id, a_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ v, bv, ak1.a._row_id, ak1.k1, a.k1 ], primary key: [ $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3 ], read pk prefix len hint: 3 } @@ -1049,12 +940,12 @@ └── StreamExchange Hash([0]) from 2 Fragment 1 - Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } + Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── Upstream └── BatchPlanNode 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: 6 } + Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } ├── Upstream └── BatchPlanNode @@ -1068,22 +959,6 @@ Table 4 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 - ├── columns: [ vnode, _row_id, a_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 6 - ├── columns: [ vnode, k1, a__row_id, ak1_backfill_finished ] - ├── 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: [ v, bv, a.k1, ak1.a._row_id ] ├── primary key: [ $2 ASC, $3 ASC ] @@ -1149,16 +1024,16 @@ ├── right degree table: 3 ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { result table: 4, state tables: [], distinct tables: [] } │ └── StreamExchange Hash([0]) from 1 - └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { result table: 6, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { result table: 5, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 2 Fragment 1 - Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } + Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } { state table: 7 } + Chain { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } ├── Upstream └── BatchPlanNode @@ -1172,23 +1047,7 @@ Table 4 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 - ├── columns: [ vnode, _row_id, a_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 6 { columns: [ b_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 7 - ├── columns: [ vnode, _row_id, b_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 + Table 5 { columns: [ b_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 ├── columns: [ num, bv, a.k1, b.k1 ] @@ -1224,11 +1083,9 @@ ├── materialized table: 4294967294 └── StreamHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: [t1.row_id, t1.uid, t1.v, t1.created_at, window_start, window_end, t1._row_id] } └── StreamFilter { predicate: IsNotNull(t1.created_at) } - └── Chain { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } + └── Chain { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, t1_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ row_id, uid, v, created_at, window_start, window_end, t1._row_id ], primary key: [ $6 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 3 } diff --git a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml index a0381475b6bcc..5e8bebda0fbbe 100644 --- a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml +++ b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml @@ -111,20 +111,12 @@ Fragment 1 StreamProject { exprs: [TumbleStart(t.a, '01:00:00':Interval) as $expr1, t.b, t._row_id], output_watermarks: [$expr1] } - └── Chain { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 2 } + └── Chain { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── Upstream └── BatchPlanNode Table 1 { columns: [ $expr1, max(t_b), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ window_start, max ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml index 754edc5bf9ede..8de1d130800bd 100644 --- a/src/frontend/planner_test/tests/testdata/output/except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/except.yaml @@ -46,12 +46,12 @@ └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 5 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 6 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } ├── Upstream └── BatchPlanNode @@ -85,22 +85,6 @@ ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 - ├── columns: [ vnode, _row_id, t1_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 6 - ├── columns: [ vnode, _row_id, t2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ a, b, 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 } - sql: | @@ -141,12 +125,12 @@ └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } ├── Upstream └── BatchPlanNode @@ -170,22 +154,6 @@ ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 - ├── columns: [ vnode, a, t1_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 6 - ├── columns: [ vnode, a, t2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ a, b, 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 } - sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml index 101d897daa60c..b6e16f5fee294 100644 --- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml +++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml @@ -46,12 +46,12 @@ └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 5 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 6 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } ├── Upstream └── BatchPlanNode @@ -85,22 +85,6 @@ ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 - ├── columns: [ vnode, _row_id, t1_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 6 - ├── columns: [ vnode, _row_id, t2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ a, b, 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 } - sql: | @@ -141,12 +125,12 @@ └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } ├── Upstream └── BatchPlanNode @@ -170,22 +154,6 @@ ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 - ├── columns: [ vnode, a, t1_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 6 - ├── columns: [ vnode, a, t2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ a, b, 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 } - sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index 38ea1f5161ace..7f757fb18101b 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -656,17 +656,17 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 8 } + Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 9 } + Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 10 } + Chain { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── Upstream └── BatchPlanNode @@ -686,11 +686,5 @@ Table 7 { columns: [ t_src, t__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ vnode, _row_id, t_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 9 { columns: [ vnode, _row_id, t_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 10 { columns: [ vnode, _row_id, t_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], 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 } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 069d50a488946..30ffeaf761811 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -55,18 +55,9 @@ StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: "NoCheck" } ├── materialized table: 4294967294 └── 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 - Table 0 - ├── columns: [ vnode, _row_id, bid_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ auction, bidder, price, date_time, bid._row_id ] ├── primary key: [ $4 ASC ] @@ -104,18 +95,9 @@ ├── materialized table: 4294967294 └── StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price::Decimal) as $expr1, bid.date_time, 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 - Table 0 - ├── columns: [ vnode, _row_id, bid_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ auction, bidder, price, date_time, bid._row_id ] ├── primary key: [ $4 ASC ] @@ -145,12 +127,10 @@ Fragment 0 StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: "NoCheck" } { materialized table: 4294967294 } └── StreamFilter { predicate: (((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR (bid.auction = 2001:Int32)) OR (bid.auction = 2019:Int32)) OR (bid.auction = 2087:Int32)) } - └── Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ auction, price, bid._row_id ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - id: nexmark_q3 @@ -195,13 +175,13 @@ Fragment 1 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 } + └── Chain { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── Upstream └── BatchPlanNode Fragment 2 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 } + └── Chain { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } ├── Upstream └── BatchPlanNode @@ -213,10 +193,6 @@ Table 3 { columns: [ person_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, id, auction_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 5 { columns: [ vnode, id, person_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 ├── columns: [ name, city, state, id, auction.seller, person.id ] ├── primary key: [ $3 ASC, $5 ASC, $4 ASC ] @@ -292,13 +268,11 @@ Fragment 2 Chain { table: auction, columns: [auction.id, auction.date_time, auction.expires, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - ├── state table: 7 ├── Upstream └── BatchPlanNode Fragment 3 Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - ├── state table: 8 ├── Upstream └── BatchPlanNode @@ -346,22 +320,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 7 - ├── columns: [ vnode, id, auction_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 8 - ├── columns: [ vnode, _row_id, bid_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ category, avg ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q5 @@ -460,7 +418,7 @@ └── StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([1]) from 1 └── StreamProject { exprs: [window_start, max(count)] } - └── StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } { result table: 7, state tables: [ 6 ], distinct tables: [] } + └── StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } { result table: 6, state tables: [ 5 ], distinct tables: [] } └── StreamExchange Hash([1]) from 4 Fragment 1 @@ -474,7 +432,7 @@ Fragment 3 StreamHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: [bid.auction, window_start, bid._row_id] } └── StreamFilter { predicate: IsNotNull(bid.date_time) } - └── Chain { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 5 } + └── Chain { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -492,11 +450,9 @@ Table 4 { columns: [ bid_auction, window_start, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 5 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 6 { columns: [ window_start, count, bid_auction ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ window_start, count, bid_auction ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ auction, num, window_start, window_start#1 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } @@ -574,18 +530,18 @@ └── StreamExchange Hash([1]) from 2 Fragment 1 - 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: 4 } + 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) } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [$expr1, max(bid.price), ($expr1 - '00:00:10':Interval) as $expr2] } - └── StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(bid.price), count] } { result table: 5, state tables: [], distinct tables: [] } + └── StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [max(bid.price), count] } { result table: 4, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 3 Fragment 3 StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, bid.price, bid._row_id] } - └── Chain { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + └── Chain { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -597,11 +553,7 @@ Table 3 { columns: [ max(bid_price), $expr1, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 5 { columns: [ $expr1, max(bid_price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 6 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ $expr1, max(bid_price), 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, bid._row_id, $expr1 ] @@ -677,20 +629,20 @@ └── StreamHashJoin { type: Inner, predicate: person.id = auction.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 └── StreamProject { exprs: [auction.seller, $expr3, $expr4] } - └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { result table: 6, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { result table: 5, state tables: [], distinct tables: [] } └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 StreamProject { exprs: [person.id, person.name, $expr1, $expr2] } └── StreamHashAgg { group_key: [person.id, person.name, $expr1, $expr2], aggs: [count] } { result table: 4, state tables: [], distinct tables: [] } └── StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr2] } - └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } + └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4, auction.id] } - └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } + └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── Upstream └── BatchPlanNode @@ -704,11 +656,7 @@ Table 4 { columns: [ person_id, person_name, $expr1, $expr2, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 4 } - Table 5 { columns: [ vnode, id, person_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], 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: [ vnode, id, auction_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { 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 4294967294 { columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 7 } @@ -770,12 +718,11 @@ Fragment 1 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 └── BatchPlanNode Fragment 2 - 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: 6 } + 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) } ├── Upstream └── BatchPlanNode @@ -799,10 +746,6 @@ Table 4 { 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 5 { columns: [ vnode, id, auction_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 6 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id ] ├── primary key: [ $0 ASC ] @@ -834,12 +777,10 @@ StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: "NoCheck" } ├── materialized table: 4294967294 └── StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr2, 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 } + └── 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) } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 ├── columns: [ auction, bidder, price, date_time, date, time, bid._row_id ] ├── primary key: [ $6 ASC ] @@ -943,12 +884,10 @@ StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: "NoCheck" } { materialized table: 4294967294 } └── StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, bid.date_time, bid.extra, bid._row_id] } └── StreamFilter { predicate: ((0.908:Decimal * bid.price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * bid.price::Decimal) < 50000000:Decimal) } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, bid._row_id ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } - id: nexmark_q15 @@ -997,7 +936,7 @@ Fragment 1 StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, 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: 3 } + └── 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) } ├── Upstream └── BatchPlanNode @@ -1012,8 +951,6 @@ Table 2 { columns: [ $expr1, bid_auction, count_for_agg_call_8, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q16 @@ -1064,7 +1001,7 @@ Fragment 1 StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 3 } + └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -1079,8 +1016,6 @@ Table 2 { columns: [ bid_channel, $expr1, bid_auction, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, count_for_agg_call_12 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4, 5, 6 ], distribution key: [ 0, 1 ], read pk prefix len hint: 3 } - Table 3 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: nexmark_q17 @@ -1126,7 +1061,7 @@ Fragment 1 StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr1, bid.price, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 1 } + └── Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -1137,8 +1072,6 @@ ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: nexmark_q18 @@ -1175,7 +1108,6 @@ Fragment 1 Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1186,8 +1118,6 @@ ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 ├── columns: [ auction, bidder, price, channel, url, date_time, extra, bid._row_id ] ├── primary key: [ $1 ASC, $0 ASC ] @@ -1233,7 +1163,6 @@ Fragment 2 Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1244,8 +1173,6 @@ ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 ├── columns: [ auction, bidder, price, channel, url, date_time, extra, bid._row_id ] ├── primary key: [ $7 ASC ] @@ -1303,13 +1230,13 @@ └── StreamExchange Hash([0]) from 2 Fragment 1 - 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 } + 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) } ├── Upstream └── BatchPlanNode Fragment 2 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 } + └── 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) } ├── Upstream └── BatchPlanNode @@ -1321,10 +1248,6 @@ Table 3 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 5 { columns: [ vnode, id, auction_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 ├── 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, $15 ASC, $0 ASC ] @@ -1381,12 +1304,10 @@ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: "NoCheck" } { materialized table: 4294967294 } └── StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr1, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr2, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr3, bid._row_id] } - └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } + └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id ], primary key: [ $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 7 ], read pk prefix len hint: 1 } - id: nexmark_q101 @@ -1437,16 +1358,16 @@ ├── right degree table: 3 ├── StreamExchange Hash([0]) from 1 └── StreamProject { exprs: [bid.auction, max(bid.price)] } - └── StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [max(bid.price), count] } { result table: 5, state tables: [], distinct tables: [] } + └── StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [max(bid.price), count] } { result table: 4, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 2 Fragment 1 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } + Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -1458,11 +1379,7 @@ Table 3 { columns: [ bid_auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, id, auction_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 5 { columns: [ bid_auction, max(bid_price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 6 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ bid_auction, max(bid_price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 ├── columns: [ auction_id, auction_item_name, current_highest_bid, bid.auction ] @@ -1534,26 +1451,26 @@ └── StreamExchange Broadcast from 3 Fragment 1 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } + Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr1] } - └── StreamSimpleAgg { aggs: [sum0(count), count(bid.auction), count] } { result table: 9, state tables: [], distinct tables: [] } + └── StreamSimpleAgg { aggs: [sum0(count), count(bid.auction), count] } { result table: 7, state tables: [], distinct tables: [] } └── StreamExchange Single from 4 Fragment 4 - StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } { result table: 10, state tables: [], distinct tables: [] } + StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } { result table: 8, 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: 11 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -1581,15 +1498,9 @@ 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 7 { columns: [ vnode, id, auction_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 8 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 9 { columns: [ sum0(count), count(bid_auction), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - - Table 10 { columns: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ sum0(count), count(bid_auction), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 11 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ bid_auction, 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, bid_count ] @@ -1647,19 +1558,18 @@ └── StreamProject { exprs: [bid.auction] } └── StreamFilter { predicate: (count >= 20:Int32) } └── StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } - ├── result table: 5 + ├── result table: 4 ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 2 Fragment 1 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - ├── state table: 4 ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -1687,28 +1597,12 @@ └── read pk prefix len hint: 1 Table 4 - ├── columns: [ vnode, id, auction_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 5 ├── columns: [ bid_auction, count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 - ├── columns: [ vnode, _row_id, bid_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ auction_id, auction_item_name ] ├── primary key: [ $0 ASC ] @@ -1765,19 +1659,18 @@ └── StreamProject { exprs: [bid.auction] } └── StreamFilter { predicate: (count < 20:Int32) } └── StreamAppendOnlyHashAgg { group_key: [bid.auction], aggs: [count] } - ├── result table: 5 + ├── result table: 4 ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 2 Fragment 1 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - ├── state table: 4 ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -1805,28 +1698,12 @@ └── read pk prefix len hint: 1 Table 4 - ├── columns: [ vnode, id, auction_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 5 ├── columns: [ bid_auction, count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 - ├── columns: [ vnode, _row_id, bid_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ auction_id, auction_item_name ] ├── primary key: [ $0 ASC ] @@ -1890,12 +1767,12 @@ └── 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: 7 } + Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } ├── Upstream └── BatchPlanNode @@ -1924,10 +1801,6 @@ 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 7 { columns: [ vnode, id, auction_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 8 { columns: [ vnode, _row_id, bid_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ auction_id, auction_item_name, bid_count ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 2 } - id: nexmark_q106 @@ -2011,13 +1884,11 @@ Fragment 2 Chain { table: auction, columns: [auction.id, auction.date_time, auction.expires], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - ├── state table: 10 ├── Upstream └── BatchPlanNode Fragment 3 Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - ├── state table: 11 ├── Upstream └── BatchPlanNode @@ -2093,21 +1964,5 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 10 - ├── columns: [ vnode, id, auction_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 11 - ├── columns: [ vnode, _row_id, bid_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ min_final ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } 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 1eb93b957cfb5..17821e2580317 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 @@ -560,7 +560,7 @@ └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - Chain { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 1 } + Chain { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } ├── Upstream └── BatchPlanNode @@ -571,14 +571,6 @@ ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ z, t.x, t.y ] ├── primary key: [ $1 ASC, $2 ASC ] diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index d897560421a7c..0c35f6304d0d9 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -262,7 +262,6 @@ Fragment 3 Chain { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -312,14 +311,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ count ] ├── primary key: [] diff --git a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml index f6e1ab0bc2962..12e94d09a8594 100644 --- a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml @@ -30,7 +30,6 @@ ├── state tables: [ 0 ] ├── distinct tables: [] └── Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -48,14 +47,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 2 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1 ] ├── primary key: [] @@ -87,7 +78,6 @@ ├── state tables: [] ├── distinct tables: [] └── Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -98,14 +88,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1 ] ├── primary key: [] @@ -137,7 +119,6 @@ ├── state tables: [] ├── distinct tables: [] └── Chain { table: s, columns: [s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -148,14 +129,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1 ] ├── primary key: [] @@ -190,7 +163,6 @@ ├── distinct tables: [] └── StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } └── Chain { table: s, columns: [s.v, s.s, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -208,14 +180,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 2 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1 ] ├── primary key: [] @@ -259,7 +223,6 @@ ├── distinct tables: [] └── StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 4 ├── Upstream └── BatchPlanNode @@ -293,14 +256,6 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 4 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -334,7 +289,6 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [max(ao.v)] } └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -345,14 +299,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -391,7 +337,6 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum(t.v)] } └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -402,14 +347,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -443,7 +380,6 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum(ao.v)] } └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -454,14 +390,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -500,7 +428,6 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [count(t.v)] } └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -511,14 +438,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -552,7 +471,6 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [count(ao.v)] } └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -563,14 +481,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -609,7 +519,6 @@ Fragment 1 StreamProject { exprs: [t.s, ',':Varchar, t.o, t._row_id] } └── Chain { table: t, columns: [t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -627,14 +536,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 2 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -668,7 +569,6 @@ Fragment 1 StreamProject { exprs: [ao.s, ',':Varchar, ao.o, ao._row_id] } └── Chain { table: ao, columns: [ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -679,14 +579,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1 ] ├── primary key: [] @@ -730,7 +622,6 @@ ├── distinct tables: [] └── StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } └── Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 4 ├── Upstream └── BatchPlanNode @@ -764,14 +655,6 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 4 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, a2 ] ├── primary key: [] @@ -805,7 +688,6 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [max(ao.v), count(ao.v)] } └── Chain { table: ao, columns: [ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -816,14 +698,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, a2 ] ├── primary key: [] @@ -862,7 +736,6 @@ Fragment 1 StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └── Chain { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -880,14 +753,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 2 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, a2 ] ├── primary key: [] @@ -921,7 +786,6 @@ Fragment 1 StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └── Chain { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -932,14 +796,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, a2 ] ├── primary key: [] @@ -978,7 +834,6 @@ Fragment 1 StreamProject { exprs: [t.v, t.s, ',':Varchar, t.o, t._row_id] } └── Chain { table: t, columns: [t.v, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 3 ├── Upstream └── BatchPlanNode @@ -1003,14 +858,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 3 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, a2 ] ├── primary key: [] @@ -1044,7 +891,6 @@ Fragment 1 StreamProject { exprs: [ao.v, ao.s, ',':Varchar, ao.o, ao._row_id] } └── Chain { table: ao, columns: [ao.v, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1055,14 +901,6 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, a2 ] ├── primary key: [] @@ -1100,7 +938,6 @@ Fragment 1 Chain { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1118,14 +955,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, t.k ] ├── primary key: [ $1 ASC ] @@ -1158,7 +987,6 @@ ├── state tables: [ 0 ] ├── distinct tables: [] └── Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1176,14 +1004,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, k, t__row_id, tk_backfill_finished ] - ├── 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: [ a1, tk.k ] ├── primary key: [ $1 ASC ] @@ -1220,7 +1040,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } { state table: 2 } + Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── Upstream └── BatchPlanNode @@ -1238,14 +1058,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1, s.k ] ├── primary key: [ $1 ASC ] @@ -1277,7 +1089,6 @@ Fragment 1 Chain { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1288,14 +1099,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, ao.k ] ├── primary key: [ $1 ASC ] @@ -1333,7 +1136,6 @@ Fragment 1 Chain { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1344,14 +1146,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, t.k ] ├── primary key: [ $1 ASC ] @@ -1384,7 +1178,6 @@ ├── state tables: [] ├── distinct tables: [] └── Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1395,14 +1188,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, k, t__row_id, tk_backfill_finished ] - ├── 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: [ a1, tk.k ] ├── primary key: [ $1 ASC ] @@ -1439,7 +1224,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } { state table: 1 } + Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── Upstream └── BatchPlanNode @@ -1450,14 +1235,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1, s.k ] ├── primary key: [ $1 ASC ] @@ -1489,7 +1266,6 @@ Fragment 1 Chain { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1500,14 +1276,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, ao.k ] ├── primary key: [ $1 ASC ] @@ -1545,7 +1313,6 @@ Fragment 1 Chain { table: t, columns: [t.k, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1556,14 +1323,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, t.k ] ├── primary key: [ $1 ASC ] @@ -1596,7 +1355,6 @@ ├── state tables: [] ├── distinct tables: [] └── Chain { table: tk, columns: [tk.k, tk.v, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1607,14 +1365,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, k, t__row_id, tk_backfill_finished ] - ├── 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: [ a1, tk.k ] ├── primary key: [ $1 ASC ] @@ -1651,7 +1401,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } { state table: 1 } + Chain { table: s, columns: [s.k, s.v, s.o, s.t._row_id], pk: [s.t._row_id], dist: Single } ├── Upstream └── BatchPlanNode @@ -1662,14 +1412,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1, s.k ] ├── primary key: [ $1 ASC ] @@ -1701,7 +1443,6 @@ Fragment 1 Chain { table: ao, columns: [ao.k, ao.v, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1712,14 +1453,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, ao.k ] ├── primary key: [ $1 ASC ] @@ -1760,7 +1493,6 @@ Fragment 1 StreamProject { exprs: [t.k, t.s, ',':Varchar, t.o, t._row_id] } └── Chain { table: t, columns: [t.k, t.o, t.s, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1778,14 +1510,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, _row_id, t_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, t.k ] ├── primary key: [ $1 ASC ] @@ -1821,7 +1545,6 @@ ├── distinct tables: [] └── StreamProject { exprs: [tk.k, tk.s, ',':Varchar, tk.o, tk.t._row_id] } └── Chain { table: tk, columns: [tk.k, tk.o, tk.s, tk.t._row_id], pk: [tk.t._row_id], dist: UpstreamHashShard(tk.k) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1839,14 +1562,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, k, t__row_id, tk_backfill_finished ] - ├── 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: [ a1, tk.k ] ├── primary key: [ $1 ASC ] @@ -1887,7 +1602,6 @@ Fragment 1 StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } └── Chain { table: s, columns: [s.k, s.o, s.s, s.t._row_id], pk: [s.t._row_id], dist: Single } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -1905,14 +1619,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ vnode, o, t__row_id, s_backfill_finished ] - ├── 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: [ a1, s.k ] ├── primary key: [ $1 ASC ] @@ -1946,7 +1652,6 @@ Fragment 1 StreamProject { exprs: [ao.k, ao.s, ',':Varchar, ao.o, ao._row_id] } └── Chain { table: ao, columns: [ao.k, ao.o, ao.s, ao._row_id], pk: [ao._row_id], dist: UpstreamHashShard(ao._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -1957,14 +1662,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 - ├── columns: [ vnode, _row_id, ao_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a1, ao.k ] ├── primary key: [ $1 ASC ] 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 7702c096f12a3..c34b01f211fc6 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -65,7 +65,6 @@ │ ├── left table: 2 │ ├── right table: 3 │ ├── Chain { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ │ ├── state table: 4 │ │ ├── Upstream │ │ └── BatchPlanNode │ └── StreamExchange Broadcast from 1 @@ -73,11 +72,11 @@ Fragment 1 StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval) as $expr1], output_watermarks: [$expr1] } - └── StreamNow { output: [now] } { state table: 5 } + └── StreamNow { output: [now] } { state table: 4 } Fragment 2 StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval) as $expr2], output_watermarks: [$expr2] } - └── StreamNow { output: [now] } { state table: 6 } + └── StreamNow { output: [now] } { state table: 5 } Table 0 { columns: [ t1_ts, t1__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } @@ -87,17 +86,9 @@ Table 3 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4 - ├── columns: [ vnode, _row_id, t1_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 + Table 4 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 5 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 6 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ ts, t1._row_id ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 7b4b6014c92ff..e5d9b36bfd7a9 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -156,14 +156,12 @@ Fragment 1 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 1 } + └── Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode Table 0 { columns: [ lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr1), sum($expr2), count(lineitem_l_quantity), count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 1 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], 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: [ l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: tpch_q2 @@ -380,87 +378,87 @@ Fragment 4 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 14 } + └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 15 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 16 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode Fragment 7 - StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost), partsupp.ps_partkey] } { left table: 17, right table: 19, left degree table: 18, right degree table: 20 } + StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost), partsupp.ps_partkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } ├── StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } - │ └── StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } { result table: 22, state tables: [ 21 ], distinct tables: [] } - │ └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 23, right table: 25, left degree table: 24, right degree table: 26 } + │ └── StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } { result table: 19, state tables: [ 18 ], distinct tables: [] } + │ └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 20, right table: 22, left degree table: 21, right degree table: 23 } │ ├── StreamExchange Hash([0]) from 8 │ └── StreamExchange Hash([0]) from 9 - └── StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost, partsupp.ps_partkey] } { left table: 45, right table: 47, left degree table: 46, right degree table: 48 } + └── StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost, partsupp.ps_partkey] } { left table: 37, right table: 39, left degree table: 38, right degree table: 40 } ├── StreamExchange Hash([0]) from 16 └── StreamExchange Hash([0]) from 17 Fragment 8 StreamProject { exprs: [part.p_partkey] } - └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { result table: 27, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { result table: 24, state tables: [], distinct tables: [] } └── StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 28 } + └── Chain { table: part, columns: [part.p_partkey, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 9 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } { left table: 29, right table: 31, left degree table: 30, right degree table: 32 } + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } { left table: 25, right table: 27, left degree table: 26, right degree table: 28 } ├── StreamExchange Hash([2]) from 10 └── StreamExchange Hash([0]) from 13 Fragment 10 - StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } { left table: 33, right table: 35, left degree table: 34, right degree table: 36 } + StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } { left table: 29, right table: 31, left degree table: 30, right degree table: 32 } ├── StreamExchange Hash([1]) from 11 └── StreamExchange Hash([0]) from 12 Fragment 11 StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } - └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 37 } + └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── Upstream └── BatchPlanNode Fragment 12 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 38 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode Fragment 13 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { left table: 39, right table: 41, left degree table: 40, right degree table: 42 } + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { left table: 33, right table: 35, left degree table: 34, right degree table: 36 } ├── StreamExchange Hash([0]) from 14 └── StreamExchange Hash([1]) from 15 Fragment 14 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 43 } + └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } ├── Upstream └── BatchPlanNode Fragment 15 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 44 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 16 StreamProject { exprs: [part.p_partkey, part.p_mfgr] } └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 49 } + └── Chain { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 17 - Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 50 } + Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── Upstream └── BatchPlanNode @@ -492,79 +490,59 @@ Table 13 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ vnode, r_regionkey, region_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 15 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 16 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 17 { columns: [ part_p_partkey, min(partsupp_ps_supplycost) ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - - Table 18 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - - Table 19 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, partsupp_ps_supplycost, partsupp_ps_partkey ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - - Table 20 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 14 { columns: [ part_p_partkey, min(partsupp_ps_supplycost) ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 21 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 22 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, partsupp_ps_supplycost, partsupp_ps_partkey ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 23 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 24 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, supplier_s_suppkey, supplier_s_nationkey, region_r_regionkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $5 ASC, $6 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 19 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 28 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 22 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, supplier_s_suppkey, supplier_s_nationkey, region_r_regionkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $5 ASC, $6 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 29 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $2 ASC, $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 23 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 31 { columns: [ nation_n_nationkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 25 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $2 ASC, $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 32 { columns: [ nation_n_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 26 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 27 { columns: [ nation_n_nationkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 28 { columns: [ nation_n_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 35 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 36 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 30 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 37 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 31 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 38 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 32 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 39 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 40 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 34 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 41 { columns: [ nation_n_nationkey, nation_n_regionkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 35 { columns: [ nation_n_nationkey, nation_n_regionkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 42 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 36 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 43 { columns: [ vnode, r_regionkey, region_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 37 { columns: [ part_p_partkey, part_p_mfgr ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 44 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 38 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 45 { columns: [ part_p_partkey, part_p_mfgr ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 39 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 46 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 47 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 48 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 49 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 50 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 40 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost) ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [], read pk prefix len hint: 8 } @@ -694,13 +672,13 @@ Fragment 4 StreamProject { exprs: [customer.c_custkey] } └── StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } - └── Chain { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 11 } + └── Chain { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 12 } + └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode @@ -708,7 +686,6 @@ StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 13 ├── Upstream └── BatchPlanNode @@ -740,12 +717,6 @@ Table 10 { columns: [ orders_o_custkey, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, c_custkey, customer_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 12 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 13 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], 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: [ l_orderkey, revenue, o_orderdate, o_shippriority ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 3 } - id: tpch_q4 @@ -835,7 +806,6 @@ StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } └── StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -843,7 +813,6 @@ StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -857,10 +826,6 @@ Table 4 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 6 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], 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: [ o_orderpriority, order_count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q5 @@ -1007,54 +972,54 @@ Fragment 3 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'MIDDLE EAST':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 9 } + └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } ├── Upstream └── BatchPlanNode Fragment 4 StreamFilter { predicate: (nation.n_nationkey = nation.n_nationkey) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 10 } + └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 5 StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND customer.c_nationkey = supplier.s_nationkey, output: [customer.c_nationkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey] } - ├── left table: 11 - ├── right table: 13 - ├── left degree table: 12 - ├── right degree table: 14 + ├── left table: 9 + ├── right table: 11 + ├── left degree table: 10 + ├── right degree table: 12 ├── StreamExchange Hash([0, 1]) from 6 └── StreamExchange Hash([0, 3]) from 9 Fragment 6 - StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } { left table: 15, right table: 17, left degree table: 16, right degree table: 18 } + StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } { left table: 13, right table: 15, left degree table: 14, right degree table: 16 } ├── StreamExchange Hash([1]) from 7 └── StreamExchange Hash([0]) from 8 Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 19 } + └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode Fragment 8 - Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 20 } + Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── Upstream └── BatchPlanNode Fragment 9 - StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } { left table: 21, right table: 23, left degree table: 22, right degree table: 24 } + StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } { left table: 17, right table: 19, left degree table: 18, right degree table: 20 } ├── StreamExchange Hash([1]) from 10 └── StreamExchange Hash([0]) from 11 Fragment 10 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 25 } + Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode Fragment 11 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 26 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode @@ -1081,41 +1046,29 @@ Table 8 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ vnode, r_regionkey, region_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 10 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 11 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, customer_c_custkey ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 12 { columns: [ orders_o_orderkey, customer_c_nationkey, customer_c_custkey, orders_o_custkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 13 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, supplier_s_suppkey ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 3 ], read pk prefix len hint: 2 } - - Table 14 { columns: [ lineitem_l_orderkey, supplier_s_nationkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 15 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 9 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, customer_c_custkey ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 16 { columns: [ orders_o_custkey, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ orders_o_orderkey, customer_c_nationkey, customer_c_custkey, orders_o_custkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 17 { columns: [ customer_c_custkey, customer_c_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, supplier_s_suppkey ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 3 ], read pk prefix len hint: 2 } - Table 18 { columns: [ customer_c_custkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ lineitem_l_orderkey, supplier_s_nationkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 19 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 20 { columns: [ vnode, c_custkey, customer_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ orders_o_custkey, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 15 { columns: [ customer_c_custkey, customer_c_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ lineitem_l_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ customer_c_custkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 24 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ lineitem_l_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ n_name, revenue ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1174,14 +1127,11 @@ └── StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Decimal) } └── Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 1 ├── Upstream └── BatchPlanNode Table 0 { columns: [ sum(sum($expr1)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], 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: [ revenue ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: tpch_q7 @@ -1337,43 +1287,43 @@ └── StreamExchange Hash([1]) from 5 Fragment 4 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 13 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 14 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 15 } + └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode Fragment 7 - StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } { left table: 16, right table: 18, left degree table: 17, right degree table: 19 } + StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } { left table: 13, right table: 15, left degree table: 14, right degree table: 16 } ├── StreamExchange Hash([1]) from 8 └── StreamExchange Hash([1]) from 11 Fragment 8 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } { left table: 20, right table: 22, left degree table: 21, right degree table: 23 } + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } { left table: 17, right table: 19, left degree table: 18, right degree table: 20 } ├── StreamExchange Hash([0]) from 9 └── StreamExchange Hash([1]) from 10 Fragment 9 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 24 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 10 - Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 25 } + Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── Upstream └── BatchPlanNode Fragment 11 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 26 } + Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode @@ -1408,33 +1358,21 @@ Table 12 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 14 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 15 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 16 { columns: [ nation_n_name, customer_c_custkey, nation_n_nationkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - - Table 17 { columns: [ customer_c_custkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 13 { columns: [ nation_n_name, customer_c_custkey, nation_n_nationkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 18 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 14 { columns: [ customer_c_custkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ orders_o_custkey, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 20 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 21 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 22 { columns: [ customer_c_custkey, customer_c_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 16 { columns: [ orders_o_custkey, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ customer_c_nationkey, customer_c_custkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 18 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ vnode, c_custkey, customer_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ customer_c_custkey, customer_c_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 26 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ customer_c_nationkey, customer_c_custkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue ], 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 } @@ -1615,65 +1553,65 @@ Fragment 4 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } - └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 13 } + └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 14 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 15 } + Chain { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── Upstream └── BatchPlanNode Fragment 7 - StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } { left table: 16, right table: 18, left degree table: 17, right degree table: 19 } + StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } { left table: 13, right table: 15, left degree table: 14, right degree table: 16 } ├── StreamExchange Hash([1]) from 8 └── StreamExchange Hash([0]) from 15 Fragment 8 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } { left table: 20, right table: 22, left degree table: 21, right degree table: 23 } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } { left table: 17, right table: 19, left degree table: 18, right degree table: 20 } ├── StreamExchange Hash([1]) from 9 └── StreamExchange Hash([1]) from 12 Fragment 9 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { left table: 24, right table: 26, left degree table: 25, right degree table: 27 } + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { left table: 21, right table: 23, left degree table: 22, right degree table: 24 } ├── StreamExchange Hash([0]) from 10 └── StreamExchange Hash([1]) from 11 Fragment 10 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 28 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 11 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 29 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode Fragment 12 - StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } { left table: 30, right table: 32, left degree table: 31, right degree table: 33 } + StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } { left table: 25, right table: 27, left degree table: 26, right degree table: 28 } ├── StreamExchange Hash([0]) from 13 └── StreamExchange Hash([1]) from 14 Fragment 13 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 34 } + └── Chain { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 14 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 35 } + Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode Fragment 15 StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 36 } + └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode @@ -1703,53 +1641,37 @@ Table 12 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ vnode, r_regionkey, region_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 14 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 15 { columns: [ vnode, c_custkey, customer_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 16 { columns: [ nation_n_name, lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 13 { columns: [ nation_n_name, lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 17 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ orders_o_orderkey, orders_o_custkey, orders_o_orderdate ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ orders_o_orderkey, orders_o_custkey, orders_o_orderdate ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 20 { columns: [ nation_n_name, supplier_s_suppkey, nation_n_nationkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - - Table 21 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 22 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, part_p_partkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $4 ASC, $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - - Table 23 { columns: [ lineitem_l_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 24 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ nation_n_name, supplier_s_suppkey, nation_n_nationkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 26 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 18 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 19 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, part_p_partkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $4 ASC, $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 28 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ lineitem_l_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 29 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 21 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 22 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 31 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 32 { columns: [ lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $1 ASC, $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 24 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 25 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 26 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 35 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 27 { columns: [ lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $1 ASC, $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 36 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 28 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ o_year, mkt_share ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1892,7 +1814,7 @@ ├── left degree table: 2 ├── right degree table: 4 ├── StreamExchange Hash([2]) from 2 - └── StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { left table: 11, right table: 13, left degree table: 12, right degree table: 14 } + └── StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { left table: 9, right table: 11, left degree table: 10, right degree table: 12 } ├── StreamExchange Hash([1]) from 5 └── StreamExchange Hash([2]) from 8 @@ -1904,44 +1826,44 @@ Fragment 3 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 9 } + └── Chain { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 4 StreamFilter { predicate: (partsupp.ps_suppkey = partsupp.ps_suppkey) } - └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 10 } + └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── Upstream └── BatchPlanNode Fragment 5 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { left table: 15, right table: 17, left degree table: 16, right degree table: 18 } + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { left table: 13, right table: 15, left degree table: 14, right degree table: 16 } ├── StreamExchange Hash([0]) from 6 └── StreamExchange Hash([1]) from 7 Fragment 6 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 19 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 7 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 20 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode Fragment 8 - StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { left table: 21, right table: 23, left degree table: 22, right degree table: 24 } + StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { left table: 17, right table: 19, left degree table: 18, right degree table: 20 } ├── StreamExchange Hash([0]) from 9 └── StreamExchange Hash([0]) from 10 Fragment 9 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 25 } + Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode Fragment 10 StreamFilter { predicate: (lineitem.l_partkey = lineitem.l_partkey) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 26 } + └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode @@ -1963,41 +1885,29 @@ Table 8 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ nation_n_name, supplier_s_suppkey, nation_n_nationkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ nation_n_name, supplier_s_suppkey, nation_n_nationkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 11 { columns: [ orders_o_orderdate, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, orders_o_orderkey, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $2 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 12 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ orders_o_orderdate, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, orders_o_orderkey, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $2 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 13 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 15 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 16 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 17 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 14 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 19 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 16 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 20 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 17 { columns: [ orders_o_orderkey, orders_o_orderdate ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ orders_o_orderkey, orders_o_orderdate ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 19 { columns: [ lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 24 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 25 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 26 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ nation, o_year, sum_profit ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } @@ -2138,31 +2048,31 @@ └── StreamExchange Hash([3]) from 4 Fragment 3 - Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 11 } + Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 12 } + Chain { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── Upstream └── BatchPlanNode Fragment 5 - StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } { left table: 13, right table: 15, left degree table: 14, right degree table: 16 } + StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } { left table: 11, right table: 13, left degree table: 12, right degree table: 14 } ├── StreamExchange Hash([0]) from 6 └── StreamExchange Hash([0]) from 7 Fragment 6 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 17 } + └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 18 } + └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode @@ -2194,21 +2104,13 @@ Table 10 { columns: [ customer_c_nationkey, customer_c_custkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 12 { columns: [ vnode, c_custkey, customer_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 13 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 14 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 15 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [], read pk prefix len hint: 7 } @@ -2380,25 +2282,24 @@ Fragment 4 Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - ├── state table: 11 ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 12 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode Fragment 6 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 13 } + └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } - └── StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } { result table: 14, state tables: [], distinct tables: [] } + └── StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } { result table: 11, state tables: [], distinct tables: [] } └── StreamExchange Single from 8 Fragment 8 @@ -2433,13 +2334,7 @@ Table 10 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 12 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 13 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 14 { columns: [ sum(sum($expr2)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 11 { columns: [ sum(sum($expr2)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 { columns: [ ps_partkey, value ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2533,14 +2428,14 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 5 } + Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } └── StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 6 } + └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode @@ -2554,10 +2449,6 @@ Table 4 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 6 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], 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: [ l_shipmode, high_line_count, low_line_count ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q13 @@ -2652,7 +2543,6 @@ Fragment 2 Chain { table: customer, columns: [customer.c_custkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -2660,7 +2550,6 @@ StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - ├── state table: 7 ├── Upstream └── BatchPlanNode @@ -2691,22 +2580,6 @@ ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 - ├── columns: [ vnode, c_custkey, customer_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 7 - ├── columns: [ vnode, o_orderkey, orders_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 { columns: [ c_count, custdist ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q14 @@ -2792,12 +2665,11 @@ StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 5 ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 6 } + Chain { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode @@ -2811,10 +2683,6 @@ Table 4 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], 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, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ promo_revenue ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: tpch_q15 @@ -2933,30 +2801,29 @@ └── StreamExchange NoShuffle from 3 Fragment 2 - 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 } + 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) } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └── StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } { result table: 9, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } { result table: 8, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 4 Fragment 4 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └── Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 10 ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [max(max(sum($expr1)))] } - └── StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } { result table: 12, state tables: [ 11 ], distinct tables: [] } + └── StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } { result table: 10, state tables: [ 9 ], distinct tables: [] } └── StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr2], aggs: [max(sum($expr1)), count] } { result table: 14, state tables: [ 13 ], distinct tables: [] } + StreamHashAgg { group_key: [$expr2], aggs: [max(sum($expr1)), count] } { result table: 12, state tables: [ 11 ], distinct tables: [] } └── StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } └── StreamExchange NoShuffle from 3 @@ -2976,19 +2843,15 @@ Table 7 { columns: [ lineitem_l_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 9 { columns: [ lineitem_l_suppkey, sum($expr1), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ lineitem_l_suppkey, sum($expr1), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ max(sum($expr1)), $expr2 ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 11 { columns: [ max(sum($expr1)), $expr2 ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 10 { columns: [ max(max(sum($expr1))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 12 { columns: [ max(max(sum($expr1))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 11 { columns: [ $expr2, sum($expr1), lineitem_l_suppkey ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 13 { columns: [ $expr2, sum($expr1), lineitem_l_suppkey ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 14 { columns: [ $expr2, max(sum($expr1)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ $expr2, max(sum($expr1)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey ], 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 } @@ -3101,20 +2964,19 @@ Fragment 3 Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - ├── state table: 10 ├── Upstream └── BatchPlanNode Fragment 4 StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 11 } + └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [supplier.s_suppkey] } └── StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } - └── Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 12 } + └── Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode @@ -3153,12 +3015,6 @@ Table 9 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 11 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 12 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ p_brand, p_type, p_size, supplier_cnt ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - id: tpch_q17 @@ -3277,12 +3133,12 @@ └── StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } { left table: 1, right table: 3, left degree table: 2, right degree table: 4 } ├── StreamExchange Hash([2]) from 2 └── StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } - └── StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } { result table: 11, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } { result table: 9, state tables: [], distinct tables: [] } └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - ├── left table: 12 - ├── right table: 14 - ├── left degree table: 13 - ├── right degree table: 15 + ├── left table: 10 + ├── right table: 12 + ├── left degree table: 11 + ├── right degree table: 13 ├── StreamExchange Hash([0]) from 5 └── StreamExchange Hash([0]) from 6 @@ -3297,29 +3153,28 @@ Fragment 3 Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 9 ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 10 } + └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [part.p_partkey] } - └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { result table: 16, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { result table: 14, state tables: [], distinct tables: [] } └── StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 17 } + └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } - └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 18 } + └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode @@ -3346,25 +3201,17 @@ Table 8 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 10 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 11 { columns: [ part_p_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 12 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ part_p_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 13 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 18 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ avg_yearly ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } @@ -3503,7 +3350,7 @@ └── StreamProject { exprs: [lineitem.l_orderkey] } └── StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Decimal) } └── StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } - └── StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } { result table: 18, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } { result table: 15, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 6 Fragment 2 @@ -3516,22 +3363,22 @@ └── StreamExchange Hash([1]) from 4 Fragment 3 - Chain { table: customer, columns: [customer.c_custkey, customer.c_name], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { state table: 15 } + Chain { table: customer, columns: [customer.c_custkey, customer.c_name], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── Upstream └── BatchPlanNode Fragment 4 - Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 16 } + Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode Fragment 5 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 17 } + Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 19 } + Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode @@ -3586,15 +3433,7 @@ Table 14 { columns: [ orders_o_custkey, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ vnode, c_custkey, customer_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 16 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 17 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 18 { columns: [ lineitem_l_orderkey, sum(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 19 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 15 { columns: [ lineitem_l_orderkey, sum(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity ], primary key: [ $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 5 } @@ -3699,13 +3538,13 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: In(lineitem.l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (lineitem.l_shipinstruct = 'DELIVER IN PERSON':Varchar) } - └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipinstruct, lineitem.l_shipmode], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 5 } + └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipinstruct, lineitem.l_shipmode], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode Fragment 3 StreamFilter { predicate: (part.p_size >= 1:Int32) } - └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_size, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 6 } + └── Chain { table: part, columns: [part.p_partkey, part.p_brand, part.p_size, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode @@ -3719,10 +3558,6 @@ Table 4 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], 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, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ revenue ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: tpch_q20 @@ -3876,61 +3711,61 @@ └── StreamExchange Hash([0]) from 3 Fragment 2 - 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 } + 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) } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 9 } + └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [partsupp.ps_suppkey, partsupp.ps_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } └── StreamFilter { predicate: ($expr1 > $expr2) } - └── StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } + └── StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } { left table: 8, right table: 10, left degree table: 9, right degree table: 11 } ├── StreamExchange Hash([0, 1]) from 5 └── StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } - └── StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } { result table: 20, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity), count] } { result table: 16, state tables: [], distinct tables: [] } └── StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - ├── left table: 21 - ├── right table: 23 - ├── left degree table: 22 - ├── right degree table: 24 + ├── left table: 17 + ├── right table: 19 + ├── left degree table: 18 + ├── right degree table: 20 ├── StreamExchange Hash([0, 1]) from 8 └── StreamExchange Hash([0, 1]) from 9 Fragment 5 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } - └── StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: all } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } + └── StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: all } { left table: 12, right table: 14, left degree table: 13, right degree table: 15 } ├── StreamExchange Hash([0]) from 6 └── StreamExchange Hash([0]) from 7 Fragment 6 - Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 18 } + Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } - └── Chain { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { state table: 19 } + └── Chain { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } ├── Upstream └── BatchPlanNode Fragment 8 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey] } - └── StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count] } { result table: 25, state tables: [], distinct tables: [] } - └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { state table: 26 } + └── StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count] } { result table: 21, state tables: [], distinct tables: [] } + └── Chain { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── Upstream └── BatchPlanNode Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } - └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 27 } + └── Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode @@ -3950,45 +3785,33 @@ Table 7 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, $expr1 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 9 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 10 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, $expr1 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 10 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, $expr2 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 11 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 12 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, $expr2 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 13 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 14 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 15 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 16 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 17 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 18 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ vnode, p_partkey, part_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 20 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, sum(lineitem_l_quantity), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 14 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 15 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 16 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, sum(lineitem_l_quantity), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 23 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 17 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 24 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 18 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 25 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 19 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 26 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 27 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 21 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 3 } @@ -4179,24 +4002,24 @@ Fragment 5 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } - └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 19 } + └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } ├── Upstream └── BatchPlanNode Fragment 6 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 20 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── Upstream └── BatchPlanNode Fragment 7 - StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } { left table: 21, right table: 23, left degree table: 22, right degree table: 24 } + StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } { left table: 19, right table: 21, left degree table: 20, right degree table: 22 } ├── StreamExchange Hash([0]) from 8 └── StreamExchange Hash([0]) from 9 Fragment 8 StreamProject { exprs: [orders.o_orderkey] } └── StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } - └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { state table: 25 } + └── Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── Upstream └── BatchPlanNode @@ -4204,12 +4027,11 @@ StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 26 ├── Upstream └── BatchPlanNode Fragment 10 - Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 27 } + Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── Upstream └── BatchPlanNode @@ -4217,7 +4039,6 @@ StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── state table: 28 ├── Upstream └── BatchPlanNode @@ -4269,25 +4090,13 @@ Table 18 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ vnode, n_nationkey, nation_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 20 { columns: [ vnode, s_suppkey, supplier_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 21 { columns: [ orders_o_orderkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 22 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 23 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 24 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 25 { columns: [ vnode, o_orderkey, orders_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ orders_o_orderkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 21 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 28 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 22 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 { columns: [ s_name, numwait ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } diff --git a/src/frontend/planner_test/tests/testdata/output/union.yaml b/src/frontend/planner_test/tests/testdata/output/union.yaml index 276d8367df1c4..af635f1b3fa28 100644 --- a/src/frontend/planner_test/tests/testdata/output/union.yaml +++ b/src/frontend/planner_test/tests/testdata/output/union.yaml @@ -28,20 +28,16 @@ Fragment 1 StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, null:Serial, 0:Int32] } - └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } + └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [t2.a, t2.b, t2.c, null:Serial, t2._row_id, 1:Int32] } - └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 1 } + └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, t1_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 1 { columns: [ vnode, _row_id, t2_backfill_finished ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ a, b, c, t1._row_id, null:Serial, 0:Int32 ], primary key: [ $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3, 4, 5 ], read pk prefix len hint: 3 } - sql: | @@ -93,14 +89,12 @@ Fragment 2 StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, null:Serial, 0:Int32] } └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - ├── state table: 1 ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t2.a, t2.b, t2.c, null:Serial, t2._row_id, 1:Int32] } └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -111,22 +105,6 @@ ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 1 - ├── columns: [ vnode, _row_id, t1_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 2 - ├── columns: [ vnode, _row_id, t2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a, b, c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] @@ -183,14 +161,12 @@ Fragment 2 StreamProject { exprs: [t1.a, t1.b, t1.c, 0:Int32] } └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } - ├── state table: 1 ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [t2.a, t2.b, t2.c, 1:Int32] } └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } - ├── state table: 2 ├── Upstream └── BatchPlanNode @@ -201,22 +177,6 @@ ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 1 - ├── columns: [ vnode, a, t1_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - - Table 2 - ├── columns: [ vnode, a, t2_backfill_finished ] - ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 0 - Table 4294967294 ├── columns: [ a, b, c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 67a3e7fd25b27..e2c688d6a8daa 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -537,7 +537,7 @@ impl dyn PlanNode { /// hook inside to do some ad-hoc thing for [`StreamTableScan`]. pub fn to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> StreamPlanPb { if let Some(stream_table_scan) = self.as_stream_table_scan() { - return stream_table_scan.adhoc_to_stream_prost(state); + return stream_table_scan.adhoc_to_stream_prost(); } if let Some(stream_share) = self.as_stream_share() { return stream_share.adhoc_to_stream_prost(state); 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 0a2337526e3c3..7379671cb6042 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -18,19 +18,15 @@ use std::rc::Rc; use itertools::Itertools; use risingwave_common::catalog::{Field, TableDesc}; -use risingwave_common::hash::VirtualNode; -use risingwave_common::types::DataType; -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, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; use crate::expr::{ExprRewriter, FunctionCall}; -use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; +use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; -use crate::TableCatalog; /// `StreamTableScan` is a virtual plan node to represent a stream table scan. It will be converted /// to chain + merge node (for upstream materialize) + batch table scan when converting to `MView` @@ -113,56 +109,6 @@ impl StreamTableScan { pub fn chain_type(&self) -> ChainType { self.chain_type } - - /// Build catalog for backfill state - /// - /// Schema - /// ------ - /// | vnode | pk | `backfill_finished` | - /// - /// key: | vnode | - /// value: | pk | `backfill_finished` - /// - /// When we update the backfill progress, - /// we update it for all vnodes. - /// "pk" here might be confusing. It refers to the - /// upstream pk which we use to track the backfill progress. - pub fn build_backfill_state_catalog( - &self, - state: &mut BuildFragmentGraphState, - ) -> TableCatalog { - let properties = self.ctx().with_options().internal_table_subset(); - let mut catalog_builder = TableCatalogBuilder::new(properties); - let upstream_schema = &self.logical.table_desc.columns; - - // We use vnode as primary key in state table. - // If `Distribution::Single`, vnode will just be `VirtualNode::default()`. - catalog_builder.add_column(&Field::with_name(VirtualNode::RW_TYPE, "vnode")); - catalog_builder.add_order_column(0, OrderType::ascending()); - - // pk columns - for col_order in self.logical.primary_key().iter() { - let col = &upstream_schema[col_order.column_index]; - catalog_builder.add_column(&Field::from(col)); - } - - // `backfill_finished` column - catalog_builder.add_column(&Field::with_name( - DataType::Boolean, - format!("{}_backfill_finished", self.table_name()), - )); - - // Reuse the state store pk (vnode) as the vnode as well. - catalog_builder.set_vnode_col_idx(0); - catalog_builder.set_dist_key_in_pk(vec![0]); - - let num_of_columns = catalog_builder.columns().len(); - catalog_builder.set_value_indices((1..num_of_columns).collect_vec()); - - catalog_builder - .build(vec![0], 1) - .with_id(state.gen_table_id_wrapped()) - } } impl_plan_tree_node_for_leaf! { StreamTableScan } @@ -209,7 +155,7 @@ impl StreamNode for StreamTableScan { } impl StreamTableScan { - pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> PbStreamNode { + pub fn adhoc_to_stream_prost(&self) -> PbStreamNode { use risingwave_pb::stream_plan::*; let stream_key = self.base.logical_pk.iter().map(|x| *x as u32).collect_vec(); @@ -259,10 +205,6 @@ impl StreamTableScan { column_ids: upstream_column_ids.clone(), }; - let catalog = self - .build_backfill_state_catalog(state) - .to_internal_table_prost(); - PbStreamNode { fields: self.schema().to_prost(), input: vec![ @@ -292,7 +234,6 @@ impl StreamTableScan { upstream_column_ids, // The table desc used by backfill executor table_desc: Some(self.logical.table_desc.to_protobuf()), - state_table: Some(catalog), })), stream_key, operator_id: self.base.id.0 as u64, diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index bea2090886ce6..679c7b4709da6 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -80,7 +80,7 @@ pub enum Distribution { /// [`Distribution::HashShard`], but may have different vnode mapping. /// /// It exists because the upstream MV can be scaled independently. So we use - /// `UpstreamHashShard` to **force an exchange to be inserted**. + /// `UpstreamHashShard` to force an exchange is inserted. /// /// Alternatively, [`Distribution::SomeShard`] can also be used to insert an exchange, but /// `UpstreamHashShard` contains distribution keys, which might be useful in some cases, e.g., diff --git a/src/frontend/src/utils/stream_graph_formatter.rs b/src/frontend/src/utils/stream_graph_formatter.rs index 0acead479274e..fd8f0e89b74a8 100644 --- a/src/frontend/src/utils/stream_graph_formatter.rs +++ b/src/frontend/src/utils/stream_graph_formatter.rs @@ -255,12 +255,6 @@ impl StreamGraphFormatter { self.pretty_add_table(node.get_state_table().unwrap()), )); } - stream_node::NodeBody::Chain(node) => { - fields.push(( - "state table", - self.pretty_add_table(node.get_state_table().unwrap()), - )) - } _ => {} }; diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index a204885281024..2971dc6f7e271 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -778,7 +778,6 @@ where &self, table_id: TableId, internal_table_ids: Vec, - fragment_manager: FragmentManagerRef, ) -> MetaResult<(NotificationVersion, Vec)> { let core = &mut *self.core.lock().await; let database_core = &mut core.database; @@ -796,28 +795,6 @@ where .map(|(index_id, index)| (*index_id, index.index_table_id)) .unzip(); - let mut index_internal_table_ids = Vec::with_capacity(index_table_ids.len()); - for index_table_id in &index_table_ids { - let internal_table_ids = match fragment_manager - .select_table_fragments_by_table_id(&(index_table_id.into())) - .await - .map(|fragments| fragments.internal_table_ids()) - { - Ok(v) => v, - // Handle backwards compat with no state persistence. - Err(_) => vec![], - }; - - // 1 should be used by table scan. - if internal_table_ids.len() == 1 { - index_internal_table_ids.push(internal_table_ids[0]); - } else { - // backwards compatibility with indexes - // without backfill state persisted. - assert_eq!(internal_table_ids.len(), 0); - } - } - if let Some(ref_count) = database_core.relation_ref_count.get(&table_id).cloned() { if ref_count > index_ids.len() { return Err(MetaError::permission_denied(format!( @@ -835,14 +812,6 @@ where .iter() .map(|index_table_id| tables.remove(*index_table_id).unwrap()) .collect_vec(); - let index_internal_tables = index_internal_table_ids - .iter() - .map(|index_internal_table_id| { - tables - .remove(*index_internal_table_id) - .expect("internal index table should exist") - }) - .collect_vec(); for index_table in &index_tables { if let Some(ref_count) = database_core.relation_ref_count.get(&index_table.id) { return Err(MetaError::permission_denied(format!( @@ -864,7 +833,6 @@ where let users_need_update = { let table_to_drop_ids = index_table_ids .iter() - .chain(&index_internal_table_ids) .chain(&internal_table_ids) .chain([&table_id]) .collect_vec(); @@ -914,7 +882,6 @@ where internal_tables .into_iter() .chain(index_tables.into_iter()) - .chain(index_internal_tables.into_iter()) .map(|internal_table| Relation { relation_info: RelationInfo::Table(internal_table).into(), }), @@ -953,7 +920,6 @@ where &self, index_id: IndexId, index_table_id: TableId, - internal_table_ids: Vec, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; @@ -980,23 +946,11 @@ where table.name, ref_count ))), None => { - let internal_tables = internal_table_ids - .iter() - .map(|internal_table_id| { - tables - .remove(*internal_table_id) - .expect("internal table should exist") - }) - .collect_vec(); - let dependent_relations = table.dependent_relations.clone(); - let objects = iter::once(table.id) - .chain(internal_table_ids) - .map(Object::TableId) - .collect_vec(); + let objects = &[Object::TableId(table.id)]; - let users_need_update = Self::update_user_privileges(&mut users, &objects); + let users_need_update = Self::update_user_privileges(&mut users, objects); commit_meta!(self, tables, indexes, users)?; @@ -1023,12 +977,7 @@ where Relation { relation_info: RelationInfo::Index(index).into(), }, - ] - .into_iter() - .chain(internal_tables.into_iter().map(|internal_table| Relation { - relation_info: RelationInfo::Table(internal_table).into(), - })) - .collect_vec(), + ], }), ) .await; @@ -1570,7 +1519,6 @@ where source_id: SourceId, table_id: TableId, internal_table_ids: Vec, - fragment_manager: FragmentManagerRef, ) -> MetaResult<(NotificationVersion, Vec)> { trace!(%source_id, %table_id, ?internal_table_ids, "drop table with source"); let core = &mut *self.core.lock().await; @@ -1603,30 +1551,6 @@ where .filter(|(_, index)| index.primary_table_id == table_id) .map(|(index_id, index)| (*index_id, index.index_table_id)) .unzip(); - - let mut index_internal_table_ids = Vec::with_capacity(index_table_ids.len()); - - for index_table_id in &index_table_ids { - let internal_table_ids = match fragment_manager - .select_table_fragments_by_table_id(&(index_table_id.into())) - .await - .map(|fragments| fragments.internal_table_ids()) - { - Ok(v) => v, - // Handle backwards compat with no state persistence. - Err(_) => vec![], - }; - - // 1 should be used by table scan. - if internal_table_ids.len() == 1 { - index_internal_table_ids.push(internal_table_ids[0]); - } else { - // backwards compatibility with indexes - // without backfill state persisted. - assert_eq!(internal_table_ids.len(), 0); - } - } - if let Some(ref_count) = database_core.relation_ref_count.get(&table_id).cloned() { // Indexes are dependent on table. We can drop table only if its `ref_count` is // strictly equal to number of indexes. @@ -1652,14 +1576,6 @@ where .iter() .map(|index_table_id| tables.remove(*index_table_id).unwrap()) .collect_vec(); - let index_internal_tables = index_internal_table_ids - .iter() - .map(|index_internal_table_id| { - tables - .remove(*index_internal_table_id) - .expect("internal index table should exist") - }) - .collect_vec(); for index_table in &index_tables { if let Some(ref_count) = database_core.relation_ref_count.get(&index_table.id) { return Err(MetaError::permission_denied(format!( @@ -1679,11 +1595,6 @@ where .into_iter() .chain(internal_table_ids.iter().map(|id| Object::TableId(*id))) .chain(index_table_ids.iter().map(|id| Object::TableId(*id))) - .chain( - index_internal_table_ids - .iter() - .map(|id| Object::TableId(*id)), - ) .collect_vec(); let users_need_update = Self::update_user_privileges(&mut users, &objects); @@ -1726,7 +1637,6 @@ where internal_tables .into_iter() .chain(index_tables.into_iter()) - .chain(index_internal_tables.into_iter()) .map(|internal_table| Relation { relation_info: RelationInfo::Table(internal_table) .into(), @@ -1815,7 +1725,6 @@ where pub async fn finish_create_index_procedure( &self, - internal_tables: Vec, index: &Index, table: &Table, ) -> MetaResult { @@ -1838,9 +1747,7 @@ where indexes.insert(index.id, index.clone()); tables.insert(table.id, table.clone()); - for table in &internal_tables { - tables.insert(table.id, table.clone()); - } + commit_meta!(self, indexes, tables)?; let version = self @@ -1854,12 +1761,7 @@ where Relation { relation_info: RelationInfo::Index(index.to_owned()).into(), }, - ] - .into_iter() - .chain(internal_tables.into_iter().map(|internal_table| Relation { - relation_info: RelationInfo::Table(internal_table).into(), - })) - .collect_vec(), + ], }), ) .await; diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index c2fb0ca5dc3fd..2b45e3c3f2c93 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -296,7 +296,7 @@ where let (version, streaming_job_ids) = match job_id { StreamingJobId::MaterializedView(table_id) => { self.catalog_manager - .drop_table(table_id, internal_table_ids, self.fragment_manager.clone()) + .drop_table(table_id, internal_table_ids) .await? } StreamingJobId::Sink(sink_id) => { @@ -307,19 +307,14 @@ where (version, vec![sink_id.into()]) } StreamingJobId::Table(source_id, table_id) => { - self.drop_table_inner( - source_id, - table_id, - internal_table_ids, - self.fragment_manager.clone(), - ) - .await? + self.drop_table_inner(source_id, table_id, internal_table_ids) + .await? } StreamingJobId::Index(index_id) => { let index_table_id = self.catalog_manager.get_index_table(index_id).await?; let version = self .catalog_manager - .drop_index(index_id, index_table_id, internal_table_ids) + .drop_index(index_id, index_table_id) .await?; (version, vec![index_table_id.into()]) } @@ -508,7 +503,7 @@ where StreamingJob::Index(index, table) => { creating_internal_table_ids.push(table.id); self.catalog_manager - .finish_create_index_procedure(internal_tables, index, table) + .finish_create_index_procedure(index, table) .await? } }; @@ -526,7 +521,6 @@ where source_id: Option, table_id: TableId, internal_table_ids: Vec, - fragment_manager: FragmentManagerRef, ) -> MetaResult<( NotificationVersion, Vec, @@ -536,12 +530,7 @@ where // `associated_source_id`. Indexes also need to be dropped atomically. let (version, delete_jobs) = self .catalog_manager - .drop_table_with_source( - source_id, - table_id, - internal_table_ids, - fragment_manager.clone(), - ) + .drop_table_with_source(source_id, table_id, internal_table_ids) .await?; // Unregister source connector worker. self.source_manager @@ -550,7 +539,7 @@ where Ok((version, delete_jobs)) } else { self.catalog_manager - .drop_table(table_id, internal_table_ids, fragment_manager) + .drop_table(table_id, internal_table_ids) .await } } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 281a6a9376727..7960b4a658e03 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -853,7 +853,7 @@ mod tests { async fn drop_materialized_views(&self, table_ids: Vec) -> MetaResult<()> { for table_id in &table_ids { self.catalog_manager - .drop_table(table_id.table_id, vec![], self.fragment_manager.clone()) + .drop_table(table_id.table_id, vec![]) .await?; } self.global_stream_manager diff --git a/src/stream/src/executor/backfill.rs b/src/stream/src/executor/backfill.rs index 3ebba2d9a5d6b..04636eda5c630 100644 --- a/src/stream/src/executor/backfill.rs +++ b/src/stream/src/executor/backfill.rs @@ -22,14 +22,10 @@ use either::Either; use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; -use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{self, OwnedRow, Row, RowExt}; -use risingwave_common::types::Datum; -use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{cmp_datum, OrderType}; use risingwave_hummock_sdk::HummockReadEpoch; @@ -40,9 +36,8 @@ use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef}; -use crate::common::table::state_table::StateTable; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{PkIndices, StreamExecutorResult, Watermark}; +use crate::executor::{PkIndices, Watermark}; use crate::task::{ActorId, CreateMviewProgress}; /// An implementation of the RFC: Use Backfill To Let Mv On Mv Stream Again.(https://github.com/risingwavelabs/rfcs/pull/13) @@ -68,13 +63,10 @@ use crate::task::{ActorId, CreateMviewProgress}; /// waiting. pub struct BackfillExecutor { /// Upstream table - upstream_table: StorageTable, + table: StorageTable, /// Upstream with the same schema with the upstream table. upstream: BoxedExecutor, - /// Internal state table for persisting state of backfill state. - state_table: Option>, - /// The column indices need to be forwarded to the downstream from the upstream and table scan. output_indices: Vec, @@ -93,11 +85,9 @@ impl BackfillExecutor where S: StateStore, { - #[allow(clippy::too_many_arguments)] pub fn new( - upstream_table: StorageTable, + table: StorageTable, upstream: BoxedExecutor, - state_table: Option>, output_indices: Vec, progress: CreateMviewProgress, schema: Schema, @@ -110,9 +100,8 @@ where pk_indices, identity: "BackfillExecutor".to_owned(), }, - upstream_table, + table, upstream, - state_table, output_indices, actor_id: progress.actor_id(), progress, @@ -122,74 +111,57 @@ where #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { - // The primary key columns, in the output columns of the upstream_table scan. - let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); - let state_len = pk_in_output_indices.len() + 2; // +1 for backfill_finished, +1 for vnode key. - - let pk_order = self.upstream_table.pk_serializer().get_order_types(); + // The primary key columns, in the output columns of the table scan. + let pk_in_output_indices = self.table.pk_in_output_indices().unwrap(); + let pk_order = self.table.pk_serializer().get_order_types(); - let upstream_table_id = self.upstream_table.table_id().table_id; + let table_id = self.table.table_id().table_id; let mut upstream = self.upstream.execute(); // Poll the upstream to get the first barrier. let first_barrier = expect_first_barrier(&mut upstream).await?; let init_epoch = first_barrier.epoch.prev; - if let Some(state_table) = self.state_table.as_mut() { - state_table.init_epoch(first_barrier.epoch); - } - - // Maintain backwards compatibility with no state table. - let is_finished = if let Some(state_table) = self.state_table.as_mut() { - Self::check_all_vnode_finished(state_table, state_len).await? - } else { - first_barrier.is_newly_added(self.actor_id) - }; + // If the barrier is a conf change of creating this mview, we follow the procedure of + // backfill. Otherwise, it means we've recovered and we can forward the upstream messages + // directly. + let to_create_mv = first_barrier.is_newly_added(self.actor_id); // If the snapshot is empty, we don't need to backfill. - // We cannot complete progress now, as we want to persist - // finished state to state store first. - // As such we will wait for next barrier. let is_snapshot_empty: bool = { - let snapshot = Self::snapshot_read(&self.upstream_table, init_epoch, None, false); + let snapshot = Self::snapshot_read(&self.table, init_epoch, None, false); pin_mut!(snapshot); snapshot.try_next().await?.unwrap().is_none() }; + let to_backfill = to_create_mv && !is_snapshot_empty; - // | backfill_is_finished | snapshot_empty | need_to_backfill | - // | t | t/f | f | - // | f | t | f | - // | f | f | t | - let to_backfill = !is_finished && !is_snapshot_empty; - - // Current position of the upstream_table storage primary key. - // `None` means it starts from the beginning. - let mut current_pos: Option = None; - - // Use these to persist state. - // They contain the backfill position, - // as well as the progress. - // However, they do not contain the vnode key at index 0. - // That is filled in when we flush the state table. - let mut current_state: Vec = vec![None; state_len]; - let mut old_state: Option> = None; + if to_create_mv && is_snapshot_empty { + // Directly finish the progress as the snapshot is empty. + self.progress.finish(first_barrier.epoch.curr); + } // The first barrier message should be propagated. yield Message::Barrier(first_barrier); - // If no need backfill, but state was still "unfinished" we need to finish it. - // So we just update the state + progress to meta at the next barrier to finish progress, - // and forward other messages. - // - // Reason for persisting on second barrier rather than first: - // We can't update meta with progress as finished until state_table - // has been updated. - // We also can't update state_table in first epoch, since state_table - // expects to have been initialized in previous epoch. + if !to_backfill { + // Forward messages directly to the downstream. + #[for_await] + for message in upstream { + if let Some(message) = Self::mapping_message(message?, &self.output_indices) { + yield message; + } + } + + return Ok(()); + } // The epoch used to snapshot read upstream mv. let mut snapshot_read_epoch = init_epoch; + // Current position of the table storage primary key. + // `None` means it starts from the beginning. + let mut current_pos: Option = None; + // Keep track of rows from the snapshot. let mut total_snapshot_processed_rows: u64 = 0; @@ -216,133 +188,102 @@ where // finished. // // Once the backfill loop ends, we forward the upstream directly to the downstream. - if to_backfill { - 'backfill_loop: loop { - let mut upstream_chunk_buffer: Vec = vec![]; - - let left_upstream = upstream.by_ref().map(Either::Left); - - let right_snapshot = pin!(Self::snapshot_read( - &self.upstream_table, - snapshot_read_epoch, - current_pos.clone(), - true - ) - .map(Either::Right),); - - // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier - // comes. - let backfill_stream = - select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| { - stream::PollNext::Left - }); - - let mut cur_barrier_snapshot_processed_rows: u64 = 0; - let mut cur_barrier_upstream_processed_rows: u64 = 0; - - #[for_await] - for either in backfill_stream { - match either { - // Upstream - Either::Left(msg) => { - match msg? { - Message::Barrier(barrier) => { - // If it is a barrier, switch snapshot and consume - // upstream buffer chunk - - // Consume upstream buffer chunk - for chunk in upstream_chunk_buffer.drain(..) { - cur_barrier_upstream_processed_rows += - chunk.cardinality() as u64; - if let Some(current_pos) = ¤t_pos { - yield Message::Chunk(Self::mapping_chunk( - Self::mark_chunk( - chunk, - current_pos, - &pk_in_output_indices, - pk_order, - ), - &self.output_indices, - )); - } - } + 'backfill_loop: loop { + let mut upstream_chunk_buffer: Vec = vec![]; - self.metrics - .backfill_snapshot_read_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]) - .inc_by(cur_barrier_snapshot_processed_rows); - - self.metrics - .backfill_upstream_output_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]) - .inc_by(cur_barrier_upstream_processed_rows); - - // Update snapshot read epoch. - snapshot_read_epoch = barrier.epoch.prev; - - self.progress.update( - barrier.epoch.curr, - snapshot_read_epoch, - total_snapshot_processed_rows, - ); - - // Persist state on barrier - Self::persist_state( - barrier.epoch, - &mut self.state_table, - false, - ¤t_pos, - &mut old_state, - &mut current_state, - ) - .await?; - - yield Message::Barrier(barrier); - // Break the for loop and start a new snapshot read stream. - break; - } - Message::Chunk(chunk) => { - // Buffer the upstream chunk. - upstream_chunk_buffer.push(chunk.compact()); - } - Message::Watermark(_) => { - // Ignore watermark during backfill. - } - } - } - // Snapshot read - Either::Right(msg) => { - match msg? { - None => { - // End of the snapshot read stream. - // We should not mark the chunk anymore, - // otherwise, we will ignore some rows - // in the buffer. Here we choose to never mark the chunk. - // Consume with the renaming stream buffer chunk without mark. - for chunk in upstream_chunk_buffer.drain(..) { - let chunk_cardinality = chunk.cardinality() as u64; - cur_barrier_snapshot_processed_rows += chunk_cardinality; - total_snapshot_processed_rows += chunk_cardinality; + let left_upstream = upstream.by_ref().map(Either::Left); + + let right_snapshot = pin!(Self::snapshot_read( + &self.table, + snapshot_read_epoch, + current_pos.clone(), + true + ) + .map(Either::Right),); + + // Prefer to select upstream, so we can stop snapshot stream as soon as the barrier + // comes. + let backfill_stream = + select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| { + stream::PollNext::Left + }); + + let mut cur_barrier_snapshot_processed_rows: u64 = 0; + let mut cur_barrier_upstream_processed_rows: u64 = 0; + + #[for_await] + for either in backfill_stream { + match either { + Either::Left(msg) => { + match msg? { + Message::Barrier(barrier) => { + // If it is a barrier, switch snapshot and consume + // upstream buffer chunk + + // Consume upstream buffer chunk + for chunk in upstream_chunk_buffer.drain(..) { + cur_barrier_upstream_processed_rows += + chunk.cardinality() as u64; + if let Some(current_pos) = ¤t_pos { yield Message::Chunk(Self::mapping_chunk( - chunk, + Self::mark_chunk( + chunk, + current_pos, + &pk_in_output_indices, + pk_order, + ), &self.output_indices, )); } - - break 'backfill_loop; } - Some(chunk) => { - // Raise the current position. - // As snapshot read streams are ordered by pk, so we can - // just use the last row to update `current_pos`. - current_pos = Self::update_pos(&chunk, &pk_in_output_indices); + self.metrics + .backfill_snapshot_read_row_count + .with_label_values(&[ + table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_snapshot_processed_rows); + + self.metrics + .backfill_upstream_output_row_count + .with_label_values(&[ + table_id.to_string().as_str(), + self.actor_id.to_string().as_str(), + ]) + .inc_by(cur_barrier_upstream_processed_rows); + + // Update snapshot read epoch. + snapshot_read_epoch = barrier.epoch.prev; + + self.progress.update( + barrier.epoch.curr, + snapshot_read_epoch, + total_snapshot_processed_rows, + ); + + yield Message::Barrier(barrier); + // Break the for loop and start a new snapshot read stream. + break; + } + Message::Chunk(chunk) => { + // Buffer the upstream chunk. + upstream_chunk_buffer.push(chunk.compact()); + } + Message::Watermark(_) => { + // Ignore watermark during backfill. + } + } + } + Either::Right(msg) => { + match msg? { + None => { + // End of the snapshot read stream. + // We need to set current_pos to the maximum value or do not + // mark the chunk anymore, otherwise, we will ignore some rows + // in the buffer. Here we choose to never mark the chunk. + // Consume with the renaming stream buffer chunk without mark. + for chunk in upstream_chunk_buffer.drain(..) { let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; total_snapshot_processed_rows += chunk_cardinality; @@ -351,6 +292,30 @@ where &self.output_indices, )); } + + // Finish backfill. + break 'backfill_loop; + } + Some(chunk) => { + // Raise the current position. + // As snapshot read streams are ordered by pk, so we can + // just use the last row to update `current_pos`. + current_pos = Some( + chunk + .rows() + .last() + .unwrap() + .1 + .project(&pk_in_output_indices) + .into_owned_row(), + ); + let chunk_cardinality = chunk.cardinality() as u64; + cur_barrier_snapshot_processed_rows += chunk_cardinality; + total_snapshot_processed_rows += chunk_cardinality; + yield Message::Chunk(Self::mapping_chunk( + chunk, + &self.output_indices, + )); } } } @@ -363,54 +328,14 @@ where "Backfill has already finished and forward messages directly to the downstream" ); - // Wait for first barrier to come after backfill is finished. - // So we can update our progress + persist the status. - while let Some(Ok(msg)) = upstream.next().await { - if let Some(msg) = Self::mapping_message(msg, &self.output_indices) { - // If not finished then we need to update state, otherwise no need. - if let Message::Barrier(barrier) = &msg && !is_finished { - // If snapshot was empty, we do not need to backfill, - // but we still need to persist the finished state. - // We currently persist it on the second barrier here rather than first. - // This is because we can't update state table in first epoch, - // since it expects to have been initialized in previous epoch - // (there's no epoch before the first epoch). - if is_snapshot_empty { - current_pos = - Self::construct_initial_finished_state(pk_in_output_indices.len()) - } - - // We will update current_pos at least once, - // since snapshot read has to be non-empty, - // Or snapshot was empty and we construct a placeholder state. - debug_assert_ne!(current_pos, None); - - Self::persist_state( - barrier.epoch, - &mut self.state_table, - true, - ¤t_pos, - &mut old_state, - &mut current_state, - ) - .await?; - self.progress.finish(barrier.epoch.curr); - yield msg; - break; - } - yield msg; - } - } - - // After progress finished + state persisted, - // we can forward messages directly to the downstream, - // as backfill is finished. + // Backfill has already finished. + // Forward messages directly to the downstream. #[for_await] for msg in upstream { if let Some(msg) = Self::mapping_message(msg?, &self.output_indices) { - if let Some(state_table) = self.state_table.as_mut() && let Message::Barrier(barrier) = &msg { - state_table.commit_no_data_expected(barrier.epoch); - } + if let Some(barrier) = msg.as_barrier() { + self.progress.finish(barrier.epoch.curr); + } yield msg; } } @@ -418,7 +343,7 @@ where #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read( - upstream_table: &StorageTable, + table: &StorageTable, epoch: u64, current_pos: Option, ordered: bool, @@ -430,7 +355,7 @@ where // has been consumed. The iter interface doesn't support // `Excluded(empty_row)` range bound, so we can simply return `None`. if current_pos.is_empty() { - assert!(upstream_table.pk_indices().is_empty()); + assert!(table.pk_indices().is_empty()); yield None; return Ok(()); } @@ -441,7 +366,7 @@ where }; // We use uncommitted read here, because we have already scheduled the `BackfillExecutor` // together with the upstream mv. - let iter = upstream_table + let iter = table .batch_iter_with_pk_bounds( HummockReadEpoch::NoWait(epoch), row::empty(), @@ -454,7 +379,7 @@ where pin_mut!(iter); while let Some(data_chunk) = iter - .collect_data_chunk(upstream_table.schema(), Some(CHUNK_SIZE)) + .collect_data_chunk(table.schema(), Some(CHUNK_SIZE)) .instrument_await("backfill_snapshot_read") .await? { @@ -498,10 +423,12 @@ where StreamChunk::new(ops, columns, Some(new_visibility.finish())) } - /// Builds a new stream chunk with `output_indices`. - fn mapping_chunk(chunk: StreamChunk, output_indices: &[usize]) -> StreamChunk { + fn mapping_chunk(chunk: StreamChunk, upstream_indices: &[usize]) -> StreamChunk { let (ops, columns, visibility) = chunk.into_inner(); - let mapped_columns = output_indices.iter().map(|&i| columns[i].clone()).collect(); + let mapped_columns = upstream_indices + .iter() + .map(|&i| columns[i].clone()) + .collect(); StreamChunk::new(ops, mapped_columns, visibility) } @@ -520,134 +447,6 @@ where } } } - - /// Schema - /// | vnode | pk | `backfill_finished` | - /// - /// For `current_pos` and `old_pos` are just pk of upstream. - /// They should be strictly increasing. - async fn persist_state( - epoch: EpochPair, - table: &mut Option>, - is_finished: bool, - current_pos: &Option, - old_state: &mut Option>, - current_state: &mut [Datum], - ) -> StreamExecutorResult<()> { - // Backwards compatibility with no state table in backfill. - let Some(table) = table else { - return Ok(()) - }; - if let Some(current_pos_inner) = current_pos { - // state w/o vnodes. - Self::build_temporary_state(current_state, is_finished, current_pos_inner); - Self::flush_data(table, epoch, old_state, current_state).await?; - *old_state = Some(current_state.into()); - } else { - table.commit_no_data_expected(epoch); - } - Ok(()) - } - - /// Flush the data - async fn flush_data( - table: &mut StateTable, - epoch: EpochPair, - old_state: &mut Option>, - current_partial_state: &mut [Datum], - ) -> StreamExecutorResult<()> { - let vnodes = table.vnodes().clone(); - if let Some(old_state) = old_state { - if old_state[1..] == current_partial_state[1..] { - table.commit_no_data_expected(epoch); - return Ok(()); - } else { - vnodes.iter_vnodes_scalar().for_each(|vnode| { - let datum = Some(vnode.into()); - current_partial_state[0] = datum.clone(); - old_state[0] = datum; - table.write_record(Record::Update { - old_row: &old_state[..], - new_row: &(*current_partial_state), - }) - }); - } - } else { - // No existing state, create a new entry. - vnodes.iter_vnodes_scalar().for_each(|vnode| { - let datum = Some(vnode.into()); - // fill the state - current_partial_state[0] = datum; - table.write_record(Record::Insert { - new_row: &(*current_partial_state), - }) - }); - } - table.commit(epoch).await - } - - // We want to avoid building a row for every vnode. - // Instead we can just modify a single row, and dispatch it to state table to write. - fn build_temporary_state(row_state: &mut [Datum], is_finished: bool, current_pos: &OwnedRow) { - row_state[1..current_pos.len() + 1].clone_from_slice(current_pos.as_inner()); - row_state[current_pos.len() + 1] = Some(is_finished.into()); - } - - fn update_pos(chunk: &StreamChunk, pk_in_output_indices: &[usize]) -> Option { - Some( - chunk - .rows() - .last() - .unwrap() - .1 - .project(pk_in_output_indices) - .into_owned_row(), - ) - } - - // TODO(kwannoel): I'm not sure if ["None" ..] encoding is appropriate - // for the case where upstream snapshot is empty, and we want to persist - // backfill state as "finished". - // Could it be confused with another case where pk position comprised of nulls? - // I don't think it will matter, - // because they both record that backfill is finished. - // We can revisit in future if necessary. - fn construct_initial_finished_state(pos_len: usize) -> Option { - Some(OwnedRow::new(vec![None; pos_len])) - } - - /// All vnodes should be persisted with status finished. - /// TODO: In the future we will support partial backfill recovery. - /// When that is done, this logic may need to be rewritten to handle - /// partially complete states per vnode. - async fn check_all_vnode_finished( - state_table: &StateTable, - state_len: usize, - ) -> StreamExecutorResult { - debug_assert!(!state_table.vnode_bitmap().is_empty()); - let vnodes = state_table.vnodes().iter_vnodes_scalar(); - let mut is_finished = true; - for vnode in vnodes { - let key: &[Datum] = &[Some(vnode.into())]; - let row = state_table.get_row(key).await?; - - // original_backfill_datum_pos = (state_len - 1) - // value indices are set, so we can -1 for the pk (a single vnode). - let backfill_datum_pos = state_len - 2; - let vnode_is_finished = if let Some(row) = row - && let Some(vnode_is_finished) = row.datum_at(backfill_datum_pos) - { - vnode_is_finished.into_bool() - } else { - false - }; - if !vnode_is_finished { - is_finished = false; - break; - } - } - Ok(is_finished) - } } impl Executor for BackfillExecutor diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index 3f7abf84fbfaa..f6eacee39bc4a 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId, TableOption}; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::plan_common::StorageTableDesc; @@ -22,7 +20,6 @@ use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; use super::*; -use crate::common::table::state_table::StateTable; use crate::executor::{BackfillExecutor, ChainExecutor, RearrangedChainExecutor}; pub struct ChainExecutorBuilder; @@ -113,12 +110,10 @@ impl ExecutorBuilder for ChainExecutorBuilder { .iter() .map(|&k| k as usize) .collect_vec(); - - let vnodes = params.vnode_bitmap.map(Arc::new); - let distribution = match &vnodes { + let distribution = match params.vnode_bitmap { Some(vnodes) => Distribution { dist_key_in_pk_indices, - vnodes: vnodes.clone(), + vnodes: vnodes.into(), }, None => Distribution::fallback(), }; @@ -138,8 +133,8 @@ impl ExecutorBuilder for ChainExecutorBuilder { let prefix_hint_len = table_desc.get_read_prefix_len_hint() as usize; let versioned = table_desc.versioned; // TODO: refactor it with from_table_catalog in the future. - let upstream_table = StorageTable::new_partial( - state_store.clone(), + let table = StorageTable::new_partial( + state_store, table_id, column_descs, column_ids, @@ -151,16 +146,10 @@ impl ExecutorBuilder for ChainExecutorBuilder { prefix_hint_len, versioned, ); - let state_table = if let Ok(table) = node.get_state_table() { - Some(StateTable::from_table_catalog(table, state_store, vnodes).await) - } else { - None - }; BackfillExecutor::new( - upstream_table, + table, mview, - state_table, output_indices, progress, schema, diff --git a/src/tests/simulation/tests/integration_tests/main.rs b/src/tests/simulation/tests/integration_tests/main.rs index 3e37baa2b99b6..db2532330ae0e 100644 --- a/src/tests/simulation/tests/integration_tests/main.rs +++ b/src/tests/simulation/tests/integration_tests/main.rs @@ -19,7 +19,6 @@ #![feature(stmt_expr_attributes)] #![cfg(madsim)] -#![feature(lazy_cell)] mod recovery; mod scale; diff --git a/src/tests/simulation/tests/integration_tests/recovery/backfill.rs b/src/tests/simulation/tests/integration_tests/recovery/backfill.rs deleted file mode 100644 index a83c217cf151d..0000000000000 --- a/src/tests/simulation/tests/integration_tests/recovery/backfill.rs +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright 2023 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::LazyLock; -use std::time::Duration; - -use anyhow::Result; -use itertools::Itertools; -use madsim::time::sleep; -use risingwave_simulation::cluster::{Cluster, Configuration, Session}; -use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; -use risingwave_simulation::utils::AssertResult; - -const ROOT_TABLE_CREATE: &str = "create table t1 (v1 int);"; -const ROOT_TABLE_DROP: &str = "drop table t1;"; -const MV1: &str = "create materialized view m1 as select * from t1 where v1 > 5;"; -const INDEX: &str = "create index i1 on t1(v1);"; -const SHOW_INTERNAL_TABLES: &str = "SHOW INTERNAL TABLES;"; - -static EXPECTED_NO_BACKFILL: LazyLock = LazyLock::new(|| { - (0..=255) - .map(|vnode| format!("{} NULL t", vnode)) - .join("\n") -}); - -fn select_all(table: impl AsRef) -> String { - format!("SELECT * FROM {} ORDER BY vnode", table.as_ref()) -} - -async fn test_no_backfill_state(session: &mut Session) -> Result<()> { - // After startup with no backfill, should be NO_BACKFILL state. - let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; - let actual = session.run(select_all(internal_table)).await?; - assert_eq!(&actual, EXPECTED_NO_BACKFILL.as_str()); - Ok(()) -} - -#[madsim::test] -async fn test_snapshot_mv() -> Result<()> { - let mut cluster = Cluster::start(Configuration::for_scale()).await?; - let mut session = cluster.start_session(); - - session.run(ROOT_TABLE_CREATE).await?; - session.run(MV1).await?; - - test_no_backfill_state(&mut session).await?; - - session - .run(&format!( - "insert into t1 values {}", - (1..=10).map(|x| format!("({x})")).join(",") - )) - .await?; - - session.run("flush").await?; - - // v1 > 5, result is [6, 7, 8, 9, 10] - session - .run("select count(*) from m1") - .await? - .assert_result_eq("5"); - - // After startup with no backfill, with data inserted after, should be NO_BACKFILL state. - test_no_backfill_state(&mut session).await?; - - let fragment = cluster - .locate_one_fragment([ - identity_contains("materialize"), - no_identity_contains("chain"), - ]) - .await?; - - let id = fragment.id(); - - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(3)).await; - - // Before complete recovery should be NO_BACKFILL state - test_no_backfill_state(&mut session).await?; - - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(3)).await; - - // After recovery should be NO_BACKFILL state - test_no_backfill_state(&mut session).await?; - - Ok(()) -} - -#[madsim::test] -async fn test_backfill_mv() -> Result<()> { - let mut cluster = Cluster::start(Configuration::for_scale()).await?; - let mut session = cluster.start_session(); - - session.run(ROOT_TABLE_CREATE).await?; - - session - .run(&format!( - "insert into t1 values {}", - (1..=10).map(|x| format!("({x})")).join(",") - )) - .await?; - session.run("flush").await?; - - session.run(MV1).await?; - - let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; - let results = session - .run(format!("SELECT * FROM {}", internal_table)) - .await?; - assert_eq!(results.lines().collect_vec().len(), 256); - - let fragment = cluster - .locate_one_fragment([ - identity_contains("materialize"), - no_identity_contains("chain"), - ]) - .await?; - - let id = fragment.id(); - - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(3)).await; - - let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; - let results = session - .run(format!("SELECT * FROM {}", internal_table)) - .await?; - assert_eq!(results.lines().collect_vec().len(), 256); - - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(3)).await; - - let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; - let results = session - .run(format!("SELECT * FROM {}", internal_table)) - .await?; - assert_eq!(results.lines().collect_vec().len(), 256); - - Ok(()) -} - -#[madsim::test] -async fn test_index_backfill() -> Result<()> { - let mut cluster = Cluster::start(Configuration::for_scale()).await?; - let mut session = cluster.start_session(); - - session.run(ROOT_TABLE_CREATE).await?; - - session - .run(&format!( - "insert into t1 values {}", - (1..=10).map(|x| format!("({x})")).join(",") - )) - .await?; - session.run("flush").await?; - - session.run(INDEX).await?; - - let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; - let results = session - .run(format!("SELECT * FROM {}", internal_table)) - .await?; - assert_eq!(results.lines().collect_vec().len(), 256); - - let fragment = cluster - .locate_one_fragment([identity_contains("index"), no_identity_contains("chain")]) - .await?; - - let id = fragment.id(); - - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(3)).await; - - let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; - let results = session - .run(format!("SELECT * FROM {}", internal_table)) - .await?; - assert_eq!(results.lines().collect_vec().len(), 256); - - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; - sleep(Duration::from_secs(3)).await; - - let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; - let results = session - .run(format!("SELECT * FROM {}", internal_table)) - .await?; - assert_eq!(results.lines().collect_vec().len(), 256); - - session.run(ROOT_TABLE_DROP).await?; - let results = session.run(SHOW_INTERNAL_TABLES).await?; - assert_eq!(results, ""); - - Ok(()) -} diff --git a/src/tests/simulation/tests/integration_tests/recovery/mod.rs b/src/tests/simulation/tests/integration_tests/recovery/mod.rs index 3b45a0d63853e..299edb092d7bd 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/mod.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/mod.rs @@ -12,5 +12,4 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod backfill; mod nexmark_recovery;