From 149fda2adb74c2ad572d358f51b307f019f3c3d6 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 13 Jan 2025 18:28:20 +0800 Subject: [PATCH] pick --- e2e_test/batch/basic/rw_timestamp.slt.part | 44 + e2e_test/ddl/show.slt | 3 + e2e_test/extended_mode/basic.slt | 2 + e2e_test/source_legacy/basic/nosim_kafka.slt | 1 + .../cdc_inline/alter/cdc_table_alter.slt | 1 + .../cdc_inline/auto_schema_change_mysql.slt | 4 + .../cdc_inline/auto_schema_change_pg.slt | 3 + .../cdc_inline/auto_schema_map_mysql.slt | 2 + .../cdc_inline/auto_schema_map_pg.slt | 1 + .../executor/join/distributed_lookup_join.rs | 2 +- src/batch/src/executor/row_seq_scan.rs | 31 +- src/common/src/catalog/column.rs | 55 +- src/common/src/catalog/mod.rs | 11 + .../codec/tests/integration_tests/utils.rs | 1 + src/connector/src/parser/plain_parser.rs | 3 + src/connector/src/source/manager.rs | 2 + .../tests/testdata/input/rw_timestamp.yaml | 22 + .../tests/testdata/output/agg.yaml | 119 +- .../tests/testdata/output/array.yaml | 2 +- .../tests/testdata/output/basic_query.yaml | 8 +- .../output/case_when_optimization.yaml | 10 +- .../tests/testdata/output/ch_benchmark.yaml | 963 ++++++++-------- .../tests/testdata/output/column_pruning.yaml | 34 +- .../output/common_table_expressions.yaml | 16 +- .../tests/testdata/output/delete.yaml | 6 +- .../testdata/output/distribution_derive.yaml | 243 ++-- .../testdata/output/emit_on_window_close.yaml | 26 +- .../tests/testdata/output/except.yaml | 37 +- .../tests/testdata/output/explain.yaml | 16 +- .../tests/testdata/output/expr.yaml | 8 +- .../output/functional_dependency.yaml | 6 +- .../testdata/output/index_selection.yaml | 4 +- .../tests/testdata/output/insert.yaml | 6 +- .../tests/testdata/output/intersect.yaml | 37 +- .../tests/testdata/output/join.yaml | 153 +-- .../tests/testdata/output/join_ordering.yaml | 32 +- .../tests/testdata/output/limit.yaml | 16 +- .../logical_scan_predicate_eliminate.yaml | 6 +- .../tests/testdata/output/nexmark.yaml | 483 ++++---- .../tests/testdata/output/nexmark_source.yaml | 438 ++++--- .../testdata/output/nexmark_source_kafka.yaml | 424 ++++--- .../output/nexmark_temporal_filter.yaml | 498 +++++--- .../testdata/output/nexmark_watermark.yaml | 360 +++--- .../testdata/output/over_window_function.yaml | 140 +-- .../testdata/output/predicate_pushdown.yaml | 44 +- .../tests/testdata/output/range_scan.yaml | 2 +- .../tests/testdata/output/rw_timestamp.yaml | 41 + .../tests/testdata/output/share.yaml | 16 +- .../tests/testdata/output/shared_views.yaml | 6 +- .../tests/testdata/output/short_circuit.yaml | 4 +- .../tests/testdata/output/sink.yaml | 10 +- .../testdata/output/stream_dist_agg.yaml | 340 ++++-- .../testdata/output/struct_field_access.yaml | 36 +- .../tests/testdata/output/struct_query.yaml | 20 +- .../tests/testdata/output/subquery.yaml | 46 +- .../tests/testdata/output/subquery_expr.yaml | 68 +- .../output/subquery_expr_correlated.yaml | 78 +- .../testdata/output/temporal_filter.yaml | 55 +- .../tests/testdata/output/time_window.yaml | 34 +- .../tests/testdata/output/tpch.yaml | 1012 +++++++++-------- .../tests/testdata/output/tpch_variant.yaml | 431 +++---- .../tests/testdata/output/union.yaml | 113 +- .../tests/testdata/output/update.yaml | 45 +- .../tests/testdata/output/watermark.yaml | 14 +- src/frontend/src/binder/expr/mod.rs | 1 + src/frontend/src/binder/update.rs | 3 +- src/frontend/src/catalog/table_catalog.rs | 32 +- .../src/handler/alter_table_column.rs | 1 + src/frontend/src/handler/create_mv.rs | 7 +- src/frontend/src/handler/create_sink.rs | 21 +- src/frontend/src/handler/create_table.rs | 6 +- src/frontend/src/handler/describe.rs | 1 + src/frontend/src/handler/drop_sink.rs | 3 +- src/frontend/src/optimizer/mod.rs | 11 +- .../src/optimizer/plan_visitor/mod.rs | 2 + .../plan_visitor/rw_timestamp_validator.rs | 47 + .../optimizer/rule/index_selection_rule.rs | 22 +- src/frontend/src/planner/delete.rs | 6 +- src/frontend/src/planner/update.rs | 2 +- src/storage/src/row_serde/mod.rs | 4 + .../src/table/batch_table/storage_table.rs | 72 +- 81 files changed, 3972 insertions(+), 2963 deletions(-) create mode 100644 e2e_test/batch/basic/rw_timestamp.slt.part create mode 100644 src/frontend/planner_test/tests/testdata/input/rw_timestamp.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/rw_timestamp.yaml create mode 100644 src/frontend/src/optimizer/plan_visitor/rw_timestamp_validator.rs diff --git a/e2e_test/batch/basic/rw_timestamp.slt.part b/e2e_test/batch/basic/rw_timestamp.slt.part new file mode 100644 index 0000000000000..ce3427f786ac7 --- /dev/null +++ b/e2e_test/batch/basic/rw_timestamp.slt.part @@ -0,0 +1,44 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table t (a int, id int primary key); + +statement ok +create index idx on t(a); + +statement ok +insert into t values (1, 1), (2, 2); + +query ?? rowsort +select ABS(EXTRACT(EPOCH FROM (_rw_timestamp - now()))) < 2, a, id from t; +---- +t 1 1 +t 2 2 + +sleep 3s + +statement ok +update t set a = 11 where id = 1; + +query ?? rowsort +select ABS(EXTRACT(EPOCH FROM (_rw_timestamp - now()))) < 2, a, id from t; +---- +f 2 2 +t 11 1 + +query ?? rowsort +select ABS(EXTRACT(EPOCH FROM (_rw_timestamp - now()))) < 2, a, id from t where id = 1; +---- +t 11 1 + +query ?? rowsort +select ABS(EXTRACT(EPOCH FROM (_rw_timestamp - now()))) < 2, a, id from t where a = 11; +---- +t 11 1 + +statement ok +delete from t; + +statement ok +drop table t; diff --git a/e2e_test/ddl/show.slt b/e2e_test/ddl/show.slt index 2a3cbf61c1651..8eece6d7c18a9 100644 --- a/e2e_test/ddl/show.slt +++ b/e2e_test/ddl/show.slt @@ -26,6 +26,7 @@ v1 integer false turpis vehicula v2 integer false Lorem ipsum dolor sit amet v3 integer false NULL _row_id serial true consectetur adipiscing elit +_rw_timestamp timestamp with time zone true NULL primary key _row_id NULL NULL distribution key _row_id NULL NULL table description t3 NULL volutpat vitae @@ -37,6 +38,7 @@ v1 integer false turpis vehicula v2 integer false Lorem ipsum dolor sit amet v3 integer false NULL _row_id serial true consectetur adipiscing elit +_rw_timestamp timestamp with time zone true NULL statement ok create index idx1 on t3 (v1,v2); @@ -65,6 +67,7 @@ v1 integer false Nemo enim ipsam v2 integer false NULL v3 integer false NULL _row_id serial true NULL +_rw_timestamp timestamp with time zone true NULL primary key _row_id NULL NULL distribution key _row_id NULL NULL idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) NULL NULL diff --git a/e2e_test/extended_mode/basic.slt b/e2e_test/extended_mode/basic.slt index 59195d093f1ce..07436820a0199 100644 --- a/e2e_test/extended_mode/basic.slt +++ b/e2e_test/extended_mode/basic.slt @@ -46,6 +46,7 @@ v1 integer false NULL v2 integer false NULL v3 integer false NULL _row_id serial true NULL +_rw_timestamp timestamp with time zone true NULL primary key _row_id NULL NULL distribution key _row_id NULL NULL table description t3 NULL NULL @@ -57,6 +58,7 @@ v1 integer false NULL v2 integer false NULL v3 integer false NULL _row_id serial true NULL +_rw_timestamp timestamp with time zone true NULL statement ok drop table t3; diff --git a/e2e_test/source_legacy/basic/nosim_kafka.slt b/e2e_test/source_legacy/basic/nosim_kafka.slt index b773126c9a7c1..e6ba769c8c808 100644 --- a/e2e_test/source_legacy/basic/nosim_kafka.slt +++ b/e2e_test/source_legacy/basic/nosim_kafka.slt @@ -51,6 +51,7 @@ productId bigint false NULL productName character varying false NULL tags character varying[] false NULL _row_id serial true NULL +_rw_timestamp timestamp with time zone true NULL primary key _row_id NULL NULL distribution key _row_id NULL NULL table description kafka_json_schema_plain NULL NULL diff --git a/e2e_test/source_legacy/cdc_inline/alter/cdc_table_alter.slt b/e2e_test/source_legacy/cdc_inline/alter/cdc_table_alter.slt index baecff00c09a4..424d4f0995a02 100644 --- a/e2e_test/source_legacy/cdc_inline/alter/cdc_table_alter.slt +++ b/e2e_test/source_legacy/cdc_inline/alter/cdc_table_alter.slt @@ -198,6 +198,7 @@ describe my_products; id integer false NULL name character varying false NULL description character varying false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description my_products NULL NULL diff --git a/e2e_test/source_legacy/cdc_inline/auto_schema_change_mysql.slt b/e2e_test/source_legacy/cdc_inline/auto_schema_change_mysql.slt index f1c94be75ccf5..db0c3c812b796 100644 --- a/e2e_test/source_legacy/cdc_inline/auto_schema_change_mysql.slt +++ b/e2e_test/source_legacy/cdc_inline/auto_schema_change_mysql.slt @@ -41,6 +41,7 @@ id bigint false NULL modified timestamp without time zone false NULL name character varying false NULL custinfo jsonb false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_customers NULL NULL @@ -66,6 +67,7 @@ name character varying false NULL custinfo jsonb false NULL v1 character varying false NULL v2 double precision false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_customers NULL NULL @@ -96,6 +98,7 @@ name character varying false NULL custinfo jsonb false NULL v1 character varying false NULL v2 double precision false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_customers NULL NULL @@ -126,6 +129,7 @@ describe rw_customers; id bigint false NULL name character varying false NULL custinfo jsonb false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_customers NULL NULL diff --git a/e2e_test/source_legacy/cdc_inline/auto_schema_change_pg.slt b/e2e_test/source_legacy/cdc_inline/auto_schema_change_pg.slt index 8619bf8117365..ae419794b7dac 100644 --- a/e2e_test/source_legacy/cdc_inline/auto_schema_change_pg.slt +++ b/e2e_test/source_legacy/cdc_inline/auto_schema_change_pg.slt @@ -55,6 +55,7 @@ v7 timestamp without time zone false NULL v8 timestamp with time zone false NULL v9 interval false NULL v10 jsonb false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_test_schema_change NULL NULL @@ -100,6 +101,7 @@ v9 interval false NULL v10 jsonb false NULL v11 character varying false NULL v12 numeric false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_test_schema_change NULL NULL @@ -145,6 +147,7 @@ v9 interval false NULL v10 jsonb false NULL v11 character varying false NULL v12 numeric false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_test_schema_change NULL NULL diff --git a/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt index 098ddfc8cf021..d9ce495f28b4b 100644 --- a/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt @@ -120,6 +120,7 @@ describe rw_customers; id bigint false NULL modified timestamp without time zone false NULL custinfo jsonb false NULL +_rw_timestamp timestamp with time zone true NULL primary key id NULL NULL distribution key id NULL NULL table description rw_customers NULL NULL @@ -153,6 +154,7 @@ c_datetime timestamp without time zone false NULL c_timestamp timestamp with time zone false NULL c_enum character varying false NULL c_json jsonb false NULL +_rw_timestamp timestamp with time zone true NULL primary key c_boolean, c_bigint, c_date NULL NULL distribution key c_boolean, c_bigint, c_date NULL NULL table description rw_mysql_types_test NULL NULL diff --git a/e2e_test/source_legacy/cdc_inline/auto_schema_map_pg.slt b/e2e_test/source_legacy/cdc_inline/auto_schema_map_pg.slt index be220347225bb..3cdfd3c2eeaec 100644 --- a/e2e_test/source_legacy/cdc_inline/auto_schema_map_pg.slt +++ b/e2e_test/source_legacy/cdc_inline/auto_schema_map_pg.slt @@ -139,6 +139,7 @@ c_interval_array interval[] false NULL c_jsonb_array jsonb[] false NULL c_uuid_array character varying[] false NULL c_enum_array character varying[] false NULL +_rw_timestamp timestamp with time zone true NULL primary key c_boolean, c_bigint, c_date NULL NULL distribution key c_boolean, c_bigint, c_date NULL NULL table description rw_postgres_types_test NULL NULL diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 0a328e2f985f1..139717e06af36 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -355,7 +355,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { let pk_prefix = OwnedRow::new(scan_range.eq_conds); - if self.lookup_prefix_len == self.table.pk_indices().len() { + if self.lookup_prefix_len == self.table.pk_indices().len() && !self.table.has_epoch_idx() { let row = self.table.get_row(&pk_prefix, self.epoch.into()).await?; if let Some(row) = row { diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 8ed9b5e9bb869..969d050be6ff7 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -408,17 +408,40 @@ impl RowSeqScanExecutor { ) -> Result> { let pk_prefix = scan_range.pk_prefix; assert!(pk_prefix.len() == table.pk_indices().len()); - let timer = histogram.as_ref().map(|histogram| histogram.start_timer()); - // Point Get. - let row = table.get_row(&pk_prefix, epoch.into()).await?; + let res = if table.has_epoch_idx() { + // has epoch_idx means we need to select `_rw_timestamp` column which is unsupported by `get_row` interface, so use iterator interface instead. + let range_bounds = (Bound::::Unbounded, Bound::Unbounded); + let iter = table + .batch_chunk_iter_with_pk_bounds( + epoch.into(), + &pk_prefix, + range_bounds, + false, + 1, + PrefetchOptions::new(false, false), + ) + .await?; + pin_mut!(iter); + let chunk = iter.next().await.transpose().map_err(BatchError::from)?; + if let Some(chunk) = chunk { + let row = chunk.row_at(0).0.to_owned_row(); + Ok(Some(row)) + } else { + Ok(None) + } + } else { + // Point Get. + let row = table.get_row(&pk_prefix, epoch.into()).await?; + Ok(row) + }; if let Some(timer) = timer { timer.observe_duration() } - Ok(row) + res } #[try_stream(ok = DataChunk, error = BatchError)] diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 119fbd8a7b04f..f7c9260e0d1ab 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -23,7 +23,7 @@ use risingwave_pb::plan_common::{ AdditionalColumn, ColumnDescVersion, DefaultColumnDesc, PbColumnCatalog, PbColumnDesc, }; -use super::{row_id_column_desc, USER_COLUMN_ID_OFFSET}; +use super::{row_id_column_desc, rw_timestamp_column_desc, USER_COLUMN_ID_OFFSET}; use crate::catalog::{cdc_table_name_column_desc, offset_column_desc, Field, ROW_ID_COLUMN_ID}; use crate::types::DataType; use crate::util::value_encoding::DatumToProtoExt; @@ -101,6 +101,11 @@ impl std::fmt::Display for ColumnId { } } +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +pub enum SystemColumn { + RwTimestamp, +} + #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct ColumnDesc { pub data_type: DataType, @@ -112,6 +117,9 @@ pub struct ColumnDesc { pub description: Option, pub additional_column: AdditionalColumn, pub version: ColumnDescVersion, + /// Currently the system column is used for `_rw_timestamp` only and is generated at runtime, + /// so this field is not persisted. + pub system_column: Option, } impl ColumnDesc { @@ -126,6 +134,7 @@ impl ColumnDesc { description: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, } } @@ -140,6 +149,7 @@ impl ColumnDesc { description: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, } } @@ -180,6 +190,27 @@ impl ColumnDesc { description: None, additional_column: additional_column_type, version: ColumnDescVersion::Pr13707, + system_column: None, + } + } + + pub fn named_with_system_column( + name: impl Into, + column_id: ColumnId, + data_type: DataType, + system_column: SystemColumn, + ) -> ColumnDesc { + ColumnDesc { + data_type, + column_id, + name: name.into(), + field_descs: vec![], + type_name: String::new(), + generated_or_default_column: None, + description: None, + additional_column: AdditionalColumn { column_type: None }, + version: ColumnDescVersion::Pr13707, + system_column: Some(system_column), } } @@ -229,6 +260,7 @@ impl ColumnDesc { description: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, } } @@ -252,6 +284,7 @@ impl ColumnDesc { description: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, } } @@ -270,6 +303,7 @@ impl ColumnDesc { generated_or_default_column: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, } } @@ -314,6 +348,7 @@ impl From for ColumnDesc { description: prost.description.clone(), additional_column, version, + system_column: None, } } } @@ -372,6 +407,10 @@ impl ColumnCatalog { self.column_desc.is_generated() } + pub fn can_dml(&self) -> bool { + !self.is_generated() && !self.is_rw_timestamp_column() + } + /// If the column is a generated column pub fn generated_expr(&self) -> Option<&ExprNode> { if let Some(GeneratedOrDefaultColumn::GeneratedColumn(desc)) = @@ -424,6 +463,20 @@ impl ColumnCatalog { } } + pub fn rw_timestamp_column() -> Self { + Self { + column_desc: rw_timestamp_column_desc(), + is_hidden: true, + } + } + + pub fn is_rw_timestamp_column(&self) -> bool { + matches!( + self.column_desc.system_column, + Some(SystemColumn::RwTimestamp) + ) + } + pub fn offset_column() -> Self { Self { column_desc: offset_column_desc(), diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 86c6e8895c066..13d89e45a4479 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -120,6 +120,17 @@ pub fn row_id_column_desc() -> ColumnDesc { ColumnDesc::named(ROWID_PREFIX, ROW_ID_COLUMN_ID, DataType::Serial) } +pub const RW_TIMESTAMP_COLUMN_NAME: &str = "_rw_timestamp"; +pub const RW_TIMESTAMP_COLUMN_ID: ColumnId = ColumnId::new(-1); +pub fn rw_timestamp_column_desc() -> ColumnDesc { + ColumnDesc::named_with_system_column( + RW_TIMESTAMP_COLUMN_NAME, + RW_TIMESTAMP_COLUMN_ID, + DataType::Timestamptz, + SystemColumn::RwTimestamp, + ) +} + pub const OFFSET_COLUMN_NAME: &str = "_rw_offset"; // The number of columns output by the cdc source job diff --git a/src/connector/codec/tests/integration_tests/utils.rs b/src/connector/codec/tests/integration_tests/utils.rs index 889dbeffc306f..620a0e9a9b2ce 100644 --- a/src/connector/codec/tests/integration_tests/utils.rs +++ b/src/connector/codec/tests/integration_tests/utils.rs @@ -176,6 +176,7 @@ impl<'a> std::fmt::Debug for ColumnDescTestDisplay<'a> { description, additional_column: AdditionalColumn { column_type }, version: _, + system_column: _, } = &self.0; write!( diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 5cbacac8c7268..86a1d3d831b6c 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -526,6 +526,7 @@ mod tests { column_type: None, }, version: Pr13707, + system_column: None, }, is_hidden: false, }, @@ -542,6 +543,7 @@ mod tests { column_type: None, }, version: Pr13707, + system_column: None, }, is_hidden: false, }, @@ -558,6 +560,7 @@ mod tests { column_type: None, }, version: Pr13707, + system_column: None, }, is_hidden: false, }, diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 67826129c8b82..cdba8ea24a33b 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -136,6 +136,7 @@ impl From<&ColumnDesc> for SourceColumnDesc { type_name: _, description: _, version: _, + system_column: _, }: &ColumnDesc, ) -> Self { if let Some(option) = generated_or_default_column { @@ -185,6 +186,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { generated_or_default_column: None, description: None, version: ColumnDescVersion::Pr13707, + system_column: None, } } } diff --git a/src/frontend/planner_test/tests/testdata/input/rw_timestamp.yaml b/src/frontend/planner_test/tests/testdata/input/rw_timestamp.yaml new file mode 100644 index 0000000000000..1dbecbe12479f --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/rw_timestamp.yaml @@ -0,0 +1,22 @@ +- sql: | + create table t (a int); + select _rw_timestamp from t; + expected_outputs: + - logical_plan + - batch_plan + - stream_error +- sql: | + create table t (a int); + select t.*, _rw_timestamp from t; + expected_outputs: + - logical_plan + - batch_plan + - stream_error +- sql: | + create table t (a int); + create index idx on t(a); + select _rw_timestamp from t where a = 1; + expected_outputs: + - logical_plan + - batch_plan + - stream_error \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 4fd9ef3718c2b..57007ada45e3a 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -116,7 +116,7 @@ LogicalProject { exprs: [min(t.v1), sum($expr1)] } └─LogicalAgg { group_key: [$expr1], aggs: [min(t.v1), sum($expr1)] } └─LogicalProject { exprs: [(t.v1 + t.v2) as $expr1, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: test logical_agg with complex group expression sql: | create table t(v1 int, v2 int, v3 int); @@ -125,7 +125,7 @@ LogicalProject { exprs: [t.v1, sum($expr2)] } └─LogicalAgg { group_key: [$expr1, t.v1], aggs: [sum($expr2)] } └─LogicalProject { exprs: [((t.v1 + t.v2) / t.v3) as $expr1, t.v1, (t.v1 * t.v2) as $expr2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } - name: test logical_agg with complex group expression sql: | create table t(v1 int, v2 int); @@ -134,7 +134,7 @@ LogicalProject { exprs: [$expr1] } └─LogicalAgg { group_key: [$expr1], aggs: [] } └─LogicalProject { exprs: [(t.v1 + t.v2) as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: "test logical_agg with complex group expression \nshould complain about nested agg call \n" sql: | create table t(v1 int, v2 int); @@ -150,7 +150,7 @@ LogicalProject { exprs: [(t.v1 + t.v2) as $expr1] } └─LogicalAgg { group_key: [t.v1, t.v2], aggs: [] } └─LogicalProject { exprs: [t.v1, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - sql: | create table t(v1 int, v2 int); select v1 from t group by v1 + v2; @@ -229,7 +229,7 @@ LogicalProject { exprs: [(4:Int32 + 5:Int32) as $expr1, (2:Int32 + 3:Int32) as $expr2] } └─LogicalAgg { group_key: [t.a], aggs: [] } └─LogicalProject { exprs: [t.a] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - name: group by output column name expr disallowed sql: | select 4 + 5 as a group by a + 1; @@ -437,7 +437,7 @@ └─LogicalFilter { predicate: (t.v1 > 5:Int32::Float64) } └─LogicalAgg { group_key: [t.v1], aggs: [] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } - name: having with non-group column sql: | create table t (v1 real, v2 int); @@ -450,7 +450,7 @@ logical_plan: |- LogicalAgg { group_key: [t.v1], aggs: [] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [v1], stream_key: [v1], pk_columns: [v1], pk_conflict: NoCheck } └─StreamProject { exprs: [t.v1], noop_update_hint: true } @@ -466,7 +466,7 @@ └─LogicalProject { exprs: [sum(t.v1)] } └─LogicalAgg { group_key: [t.v2], aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v2, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [sum], stream_key: [sum], pk_columns: [sum], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(t.v1)], noop_update_hint: true } @@ -484,7 +484,7 @@ LogicalProject { exprs: [t.v1, t.v2] } └─LogicalTopN { order: [t.v3 ASC, t.v1 ASC], limit: 1, offset: 0, group_key: [t.v1, t.v3] } └─LogicalProject { exprs: [t.v1, t.v2, t.v3] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } batch_plan: |- BatchProject { exprs: [t.v1, t.v2] } └─BatchExchange { order: [t.v3 ASC, t.v1 ASC], dist: Single } @@ -500,7 +500,7 @@ LogicalProject { exprs: [t.v1, t.v2] } └─LogicalTopN { order: [t.v1 ASC, t.v3 ASC], limit: 1, offset: 0, group_key: [t.v1, t.v3] } └─LogicalProject { exprs: [t.v1, t.v2, t.v3] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } batch_plan: |- BatchProject { exprs: [t.v1, t.v2] } └─BatchExchange { order: [t.v1 ASC, t.v3 ASC], dist: Single } @@ -515,7 +515,7 @@ logical_plan: |- LogicalTopN { order: [t.v1 ASC], limit: 1, offset: 0, group_key: [t.v1] } └─LogicalProject { exprs: [t.v1, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: distinct on sql: | create table t (v1 int, v2 int, v3 int); @@ -524,7 +524,7 @@ LogicalProject { exprs: [$expr1] } └─LogicalTopN { order: [t.v1 ASC], limit: 1, offset: 0, group_key: [t.v1] } └─LogicalProject { exprs: [(t.v2 + t.v3) as $expr1, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } batch_plan: |- BatchProject { exprs: [$expr1] } └─BatchExchange { order: [t.v1 ASC], dist: Single } @@ -541,7 +541,7 @@ LogicalProject { exprs: [count(t.v3), min(t.v2), max(t.v1)] } └─LogicalAgg { aggs: [count(t.v3), min(t.v2), max(t.v1)] } └─LogicalProject { exprs: [t.v3, t.v2, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count(t.v3), min(t.v2), max(t.v1)] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } @@ -558,7 +558,7 @@ LogicalProject { exprs: [(min(t.v1) + (max(t.v3) * count(t.v2))) as $expr1] } └─LogicalAgg { aggs: [min(t.v1), max(t.v3), count(t.v2)] } └─LogicalProject { exprs: [t.v1, t.v3, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(min(t.v1) + (max(t.v3) * count(t.v2))) as $expr1] } └─LogicalAgg { aggs: [min(t.v1), max(t.v3), count(t.v2)] } @@ -585,7 +585,7 @@ LogicalProject { exprs: [t.v1] } └─LogicalAgg { group_key: [t.v1], aggs: [] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [t.v1], aggs: [] } └─LogicalScan { table: t, columns: [t.v1] } @@ -603,7 +603,7 @@ LogicalProject { exprs: [t.v2, min(t.v1), t.v3, max(t.v1)] } └─LogicalAgg { group_key: [t.v3, t.v2], aggs: [min(t.v1), max(t.v1)] } └─LogicalProject { exprs: [t.v3, t.v2, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.v2, min(t.v1), t.v3, max(t.v1)] } └─LogicalAgg { group_key: [t.v2, t.v3], aggs: [min(t.v1), max(t.v1)] } @@ -622,7 +622,7 @@ LogicalProject { exprs: [sum(t.v1)] } └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } @@ -641,7 +641,7 @@ LogicalProject { exprs: [sum(t.v1)] } └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } @@ -660,7 +660,7 @@ LogicalProject { exprs: [sum(t.v1)] } └─LogicalAgg { aggs: [sum(t.v1)] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum(t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } @@ -679,7 +679,7 @@ LogicalProject { exprs: [sum(t.v1) filter((t.v1 > 0:Int32))] } └─LogicalAgg { aggs: [sum(t.v1) filter((t.v1 > 0:Int32))] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum(t.v1) filter((t.v1 > 0:Int32))] } └─LogicalScan { table: t, columns: [t.v1] } @@ -700,7 +700,7 @@ LogicalProject { exprs: [sum($expr1) filter(((t.a * t.b) > 0:Int32))] } └─LogicalAgg { aggs: [sum($expr1) filter(((t.a * t.b) > 0:Int32))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum($expr1) filter(((t.a * t.b) > 0:Int32))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } @@ -713,7 +713,7 @@ LogicalProject { exprs: [max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } └─LogicalAgg { aggs: [max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [max($expr1) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } @@ -734,7 +734,7 @@ LogicalProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))::Decimal) as $expr1] } └─LogicalAgg { group_key: [t.b], aggs: [sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } └─LogicalProject { exprs: [t.b, t.a] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))::Decimal) as $expr1] } └─LogicalAgg { group_key: [t.b], aggs: [sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } @@ -753,7 +753,7 @@ LogicalProject { exprs: [count filter((t.a > t.b))] } └─LogicalAgg { aggs: [count filter((t.a > t.b))] } └─LogicalProject { exprs: [t.a, t.b] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count filter((t.a > t.b))] } └─LogicalScan { table: t, columns: [t.a, t.b] } @@ -888,7 +888,7 @@ LogicalProject { exprs: [string_agg(t.y, ',':Varchar), count(distinct t.x)] } └─LogicalAgg { aggs: [string_agg(t.y, ',':Varchar), count(distinct t.x)] } └─LogicalProject { exprs: [t.y, ',':Varchar, t.x] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [string_agg(t.y, ',':Varchar), count(distinct t.x)] } └─BatchExchange { order: [], dist: Single } @@ -909,7 +909,7 @@ LogicalProject { exprs: [string_agg(t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } └─LogicalAgg { aggs: [string_agg(t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } └─LogicalProject { exprs: [t.y, ',':Varchar, t.x] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [string_agg(t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } └─BatchExchange { order: [], dist: Single } @@ -930,7 +930,7 @@ LogicalProject { exprs: [string_agg(distinct t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } └─LogicalAgg { aggs: [string_agg(distinct t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } └─LogicalProject { exprs: [t.y, ',':Varchar, t.x] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [string_agg(distinct t.y, ',':Varchar order_by(t.y ASC)), count(distinct t.x)] } └─BatchExchange { order: [], dist: Single } @@ -999,7 +999,7 @@ LogicalProject { exprs: [sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } └─LogicalAgg { aggs: [sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr1] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum($expr1) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr1] } @@ -1075,7 +1075,7 @@ LogicalAgg { aggs: [count(t.x_expanded) filter((flag = 0:Int64)), sum(t.y_expanded) filter((flag = 1:Int64))] } └─LogicalAgg { group_key: [t.x_expanded, t.y_expanded, flag], aggs: [] } └─LogicalExpand { column_subsets: [[t.x], [t.y]] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' - name: remove unnecessary distinct for max and min @@ -1227,7 +1227,7 @@ └─LogicalProjectSet { select_list: [$0, Unnest(Array(2:Int32, $1))] } └─LogicalAgg { aggs: [min(t.v1), max(t.v2)] } └─LogicalProject { exprs: [t.v1, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: post-agg project set - error sql: | create table t (v1 int, v2 int); @@ -1242,7 +1242,7 @@ └─LogicalProjectSet { select_list: [$1, Unnest(Array(2:Int32, $0))] } └─LogicalAgg { group_key: [t.v2], aggs: [min(t.v1)] } └─LogicalProject { exprs: [t.v2, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: min/max on index sql: | create table t (v1 varchar, v2 int); @@ -1252,7 +1252,7 @@ LogicalProject { exprs: [max(t.v2)] } └─LogicalAgg { aggs: [max(t.v2)] } └─LogicalProject { exprs: [t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [max(idx.v2)] } └─LogicalTopN { order: [idx.v2 DESC], limit: 1, offset: 0 } @@ -1267,7 +1267,7 @@ LogicalProject { exprs: [max(t.v2)] } └─LogicalAgg { group_key: [t.v1], aggs: [max(t.v2)] } └─LogicalProject { exprs: [t.v1, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [max(t.v2)] } └─LogicalAgg { group_key: [t.v1], aggs: [max(t.v2)] } @@ -1280,7 +1280,7 @@ LogicalProject { exprs: [min(t.v1)] } └─LogicalAgg { aggs: [min(t.v1)] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1] } + └─LogicalScan { table: t, columns: [t.v1, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [min(t.v1)] } └─LogicalTopN { order: [t.v1 ASC], limit: 1, offset: 0 } @@ -1294,7 +1294,7 @@ LogicalProject { exprs: [Case((count(t.v1) <= 1:Int32), null:Decimal, Sqrt((Greatest((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1)::Decimal)), 0:Int32::Decimal) / (count(t.v1) - 1:Int32)::Decimal))) as $expr2, Case((count(t.v1) = 0:Int32), null:Decimal, Sqrt((Greatest((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1)::Decimal)), 0:Int32::Decimal) / count(t.v1)::Decimal))) as $expr3] } └─LogicalAgg { aggs: [sum($expr1), sum(t.v1), count(t.v1)] } └─LogicalProject { exprs: [(t.v1 * t.v1) as $expr1, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } batch_plan: |- BatchProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int32), null:Decimal, Sqrt((Greatest((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1))::Decimal)), 0:Decimal) / (sum0(count(t.v1)) - 1:Int32)::Decimal))) as $expr2, Case((sum0(count(t.v1)) = 0:Int32), null:Decimal, Sqrt((Greatest((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1))::Decimal)), 0:Decimal) / sum0(count(t.v1))::Decimal))) as $expr3] } └─BatchSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1))] } @@ -1332,7 +1332,7 @@ LogicalProject { exprs: [Case((count(t.v) <= 1:Int32), null:Decimal, Sqrt((Greatest((sum($expr1)::Decimal - ((sum(t.v)::Decimal * sum(t.v)::Decimal) / count(t.v)::Decimal)), 0:Int32::Decimal) / (count(t.v) - 1:Int32)::Decimal))) as $expr2] } └─LogicalAgg { group_key: [t.w], aggs: [sum($expr1), sum(t.v), count(t.v)] } └─LogicalProject { exprs: [t.w, (t.v * t.v) as $expr1, t.v] } - └─LogicalScan { table: t, columns: [t.v, t.w, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t.w, t._row_id, t._rw_timestamp] } - name: force two phase aggregation should succeed with UpstreamHashShard and SomeShard (batch only). sql: | SET QUERY_MODE TO DISTRIBUTED; @@ -1693,19 +1693,19 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_y, t__row_id, t_x ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ t_y, t__row_id, t_x, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ first_value(t_x order_by(t_y ASC)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ first_value(t_x order_by(t_y ASC)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ first_value ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ first_value, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - sql: | create table t (x int, y int); @@ -1725,21 +1725,26 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_x ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ t_x, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ first_value(distinct t_x order_by(t_x ASC)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ first_value(distinct t_x order_by(t_x ASC)), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 2 { columns: [ t_x, count_for_agg_call_0 ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ t_x, count_for_agg_call_0, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [], read pk prefix len hint: 1 } Table 3 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ first_value ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ first_value, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - sql: | create table t (x int, y int); @@ -1868,7 +1873,7 @@ LogicalProject { exprs: [sum(t.v1), count(t.v1), sum(t.v1), count(t.v1)] } └─LogicalAgg { aggs: [sum(t.v1), count(t.v1)] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [x, y, z, w], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(sum(t.v1)), sum0(count(t.v1)), sum(sum(t.v1)), sum0(count(t.v1))] } @@ -1884,7 +1889,7 @@ LogicalProject { exprs: [approx_percentile($expr1)] } └─LogicalAgg { aggs: [approx_percentile($expr1)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [approx_percentile($expr1)] } └─BatchExchange { order: [], dist: Single } @@ -1905,7 +1910,7 @@ LogicalProject { exprs: [approx_percentile($expr1), sum(t.v1)] } └─LogicalAgg { aggs: [approx_percentile($expr1), sum(t.v1)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [approx_percentile($expr1), sum(t.v1)] } └─BatchExchange { order: [], dist: Single } @@ -1934,7 +1939,7 @@ LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1), sum(t.v1), count(t.v1)] } └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count(t.v1)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } batch_plan: |- BatchProject { exprs: [sum(t.v1), approx_percentile($expr1), sum(t.v1), count(t.v1)] } └─BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count(t.v1)] } @@ -1965,7 +1970,7 @@ LogicalProject { exprs: [approx_percentile($expr1), approx_percentile($expr1)] } └─LogicalAgg { aggs: [approx_percentile($expr1)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [approx_percentile, approx_percentile] } @@ -1982,7 +1987,7 @@ LogicalProject { exprs: [approx_percentile($expr1), approx_percentile($expr2)] } └─LogicalAgg { aggs: [approx_percentile($expr1), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [approx_percentile($expr1), approx_percentile($expr2)] } └─BatchExchange { order: [], dist: Single } @@ -2011,7 +2016,7 @@ LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1), count, (sum(t.v2)::Float64 + approx_percentile($expr2)) as $expr3] } └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchProject { exprs: [sum(t.v1), approx_percentile($expr1), count, (sum(t.v2)::Float64 + approx_percentile($expr2)) as $expr3] } └─BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } @@ -2049,7 +2054,7 @@ LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1), count, (sum(t.v2)::Float64 + approx_percentile($expr2)) as $expr3] } └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchProject { exprs: [sum(t.v1), approx_percentile($expr1), count, (sum(t.v2)::Float64 + approx_percentile($expr2)) as $expr3] } └─BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } @@ -2087,7 +2092,7 @@ LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1)] } └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1)] } └─BatchExchange { order: [], dist: Single } @@ -2116,7 +2121,7 @@ LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1), count, max(t.v2), approx_percentile($expr2)] } └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, max(t.v2), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, max(t.v2), approx_percentile($expr2)] } └─BatchExchange { order: [], dist: Single } @@ -2151,7 +2156,7 @@ LogicalProject { exprs: [approx_percentile($expr1)] } └─LogicalAgg { group_key: [t.v2], aggs: [approx_percentile($expr1)] } └─LogicalProject { exprs: [t.v2, t.v1::Float64 as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [approx_percentile($expr1)] } @@ -2182,7 +2187,7 @@ LogicalProject { exprs: [approx_percentile($expr1)] } └─LogicalAgg { aggs: [approx_percentile($expr1)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } batch_plan: |- BatchSimpleAgg { aggs: [approx_percentile($expr1)] } └─BatchExchange { order: [], dist: Single } diff --git a/src/frontend/planner_test/tests/testdata/output/array.yaml b/src/frontend/planner_test/tests/testdata/output/array.yaml index e578ddac53071..a2b9486fdb33d 100644 --- a/src/frontend/planner_test/tests/testdata/output/array.yaml +++ b/src/frontend/planner_test/tests/testdata/output/array.yaml @@ -12,7 +12,7 @@ select (ARRAY[1, v1]) from t; logical_plan: |- LogicalProject { exprs: [Array(1:Int32, t.v1) as $expr1] } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [Array(1:Int32, t.v1) as $expr1] } diff --git a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml index d9783c7f3ac24..7d43fceef8306 100644 --- a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml @@ -72,7 +72,7 @@ logical_plan: |- LogicalProject { exprs: [t.a] } └─LogicalFilter { predicate: t.a } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - name: constant folding for IS NOT TRUE, IS NOT FALSE sql: | create table t(a Boolean); @@ -80,7 +80,7 @@ logical_plan: |- LogicalProject { exprs: [t.a] } └─LogicalFilter { predicate: IsNotTrue(t.a) } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - name: constant folding IS NOT NULL sql: | create table t(a double precision); @@ -88,7 +88,7 @@ logical_plan: |- LogicalProject { exprs: [t.a] } └─LogicalFilter { predicate: IsNotNull(t.a) } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - sql: | create table t (v1 int, v2 int); select v1 from t; @@ -164,7 +164,7 @@ select * from t; logical_plan: |- LogicalProject { exprs: [] } - └─LogicalScan { table: t, columns: [t._row_id] } + └─LogicalScan { table: t, columns: [t._row_id, t._rw_timestamp] } - name: disallow subquery in values sql: | values(1, (select 1)); diff --git a/src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml b/src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml index 14156d6f5b494..48cac21d2a990 100644 --- a/src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml +++ b/src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml @@ -115,7 +115,7 @@ logical_plan: |- LogicalProject ├─exprs: ConstantLookup(t1.c1, 1:Int32, 'one':Varchar, 2:Int32, 'two':Varchar, 3:Int32, 'three':Varchar, 4:Int32, 'four':Varchar, 5:Int32, 'five':Varchar, 6:Int32, 'six':Varchar, 7:Int32, 'seven':Varchar, 8:Int32, 'eight':Varchar, 9:Int32, 'nine':Varchar, 10:Int32, 'ten':Varchar, 11:Int32, 'eleven':Varchar, 12:Int32, 'twelve':Varchar, 13:Int32, 'thirteen':Varchar, 14:Int32, 'fourteen':Varchar, 15:Int32, 'fifteen':Varchar, 16:Int32, 'sixteen':Varchar, 17:Int32, 'seventeen':Varchar, 18:Int32, 'eighteen':Varchar, 19:Int32, 'nineteen':Varchar, 20:Int32, 'twenty':Varchar, 21:Int32, 'twenty-one':Varchar, 22:Int32, 'twenty-two':Varchar, 23:Int32, 'twenty-three':Varchar, 24:Int32, 'twenty-four':Varchar, 25:Int32, 'twenty-five':Varchar, 26:Int32, 'twenty-six':Varchar, 27:Int32, 'twenty-seven':Varchar, 28:Int32, 'twenty-eight':Varchar, 29:Int32, 'twenty-nine':Varchar, 30:Int32, 'thirty':Varchar, 31:Int32, 'thirty-one':Varchar, 32:Int32, 'thirty-two':Varchar, 33:Int32, 'thirty-three':Varchar, 34:Int32, 'thirty-four':Varchar, 35:Int32, 'thirty-five':Varchar, 36:Int32, 'thirty-six':Varchar, 37:Int32, 'thirty-seven':Varchar, 38:Int32, 'thirty-eight':Varchar, 39:Int32, 'thirty-nine':Varchar, 40:Int32, 'forty':Varchar, 41:Int32, 'forty-one':Varchar, 42:Int32, 'forty-two':Varchar, 43:Int32, 'forty-three':Varchar, 44:Int32, 'forty-four':Varchar, 45:Int32, 'forty-five':Varchar, 46:Int32, 'forty-six':Varchar, 47:Int32, 'forty-seven':Varchar, 48:Int32, 'forty-eight':Varchar, 49:Int32, 'forty-nine':Varchar, 50:Int32, 'fifty':Varchar, 51:Int32, 'fifty-one':Varchar, 52:Int32, 'fifty-two':Varchar, 53:Int32, 'fifty-three':Varchar, 54:Int32, 'fifty-four':Varchar, 55:Int32, 'fifty-five':Varchar, 56:Int32, 'fifty-six':Varchar, 57:Int32, 'fifty-seven':Varchar, 58:Int32, 'fifty-eight':Varchar, 59:Int32, 'fifty-nine':Varchar, 60:Int32, 'sixty':Varchar, 61:Int32, 'sixty-one':Varchar, 62:Int32, 'sixty-two':Varchar, 63:Int32, 'sixty-three':Varchar, 64:Int32, 'sixty-four':Varchar, 65:Int32, 'sixty-five':Varchar, 66:Int32, 'sixty-six':Varchar, 67:Int32, 'sixty-seven':Varchar, 68:Int32, 'sixty-eight':Varchar, 69:Int32, 'sixty-nine':Varchar, 70:Int32, 'seventy':Varchar, 71:Int32, 'seventy-one':Varchar, 72:Int32, 'seventy-two':Varchar, 73:Int32, 'seventy-three':Varchar, 74:Int32, 'seventy-four':Varchar, 75:Int32, 'seventy-five':Varchar, 76:Int32, 'seventy-six':Varchar, 77:Int32, 'seventy-seven':Varchar, 78:Int32, 'seventy-eight':Varchar, 79:Int32, 'seventy-nine':Varchar, 80:Int32, 'eighty':Varchar, 81:Int32, 'eighty-one':Varchar, 82:Int32, 'eighty-two':Varchar, 83:Int32, 'eighty-three':Varchar, 84:Int32, 'eighty-four':Varchar, 85:Int32, 'eighty-five':Varchar, 86:Int32, 'eighty-six':Varchar, 87:Int32, 'eighty-seven':Varchar, 88:Int32, 'eighty-eight':Varchar, 89:Int32, 'eighty-nine':Varchar, 90:Int32, 'ninety':Varchar, 91:Int32, 'ninety-one':Varchar, 92:Int32, 'ninety-two':Varchar, 93:Int32, 'ninety-three':Varchar, 94:Int32, 'ninety-four':Varchar, 95:Int32, 'ninety-five':Varchar, 96:Int32, 'ninety-six':Varchar, 97:Int32, 'ninety-seven':Varchar, 98:Int32, 'ninety-eight':Varchar, 99:Int32, 'ninety-nine':Varchar, 100:Int32, 'one hundred':Varchar, '114514':Varchar) as $expr1 - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject @@ -164,7 +164,7 @@ from t1; logical_plan: |- LogicalProject { exprs: [ConstantLookup(t1.c1, 1:Int32, (t1.c1 + 1:Int32), 2:Int32, (t1.c2 + 1:Int32), 3:Int32, (t1.c3 + 1:Int32), 4:Int32, (t1.c1 + 2:Int32), 5:Int32, (t1.c2 + 2:Int32), 6:Int32, (t1.c3 + 2:Int32), 7:Int32, (t1.c1 + 3:Int32), 8:Int32, (t1.c2 + 3:Int32), 9:Int32, (t1.c3 + 3:Int32), 10:Int32, ((t1.c1 + t1.c2) + 1:Int32), 11:Int32, ((t1.c2 + t1.c3) + 2:Int32), 12:Int32, ((t1.c3 + t1.c1) + 2:Int32), 13:Int32, (t1.c1 + t1.c3), 14:Int32, (t1.c2 + t1.c2), 15:Int32, (t1.c3 + t1.c1), 16:Int32, ((t1.c1 + t1.c2) + 2:Int32), 17:Int32, ((t1.c2 + t1.c3) + 2:Int32), 18:Int32, ((t1.c3 + t1.c1) + 2:Int32), 19:Int32, (((t1.c1 + t1.c2) + t1.c3) + 1:Int32), 20:Int32, (((t1.c2 + t1.c3) + t1.c1) + 1:Int32), 21:Int32, (((t1.c3 + t1.c1) + t1.c2) + 1:Int32), 22:Int32, (((t1.c1 + t1.c2) + t1.c3) + t1.c1), 23:Int32, (((t1.c2 + t1.c3) + t1.c1) + t1.c2), 24:Int32, (((t1.c3 + t1.c1) + t1.c2) + t1.c3), 25:Int32, (t1.c1 + 1:Int32), 26:Int32, (t1.c2 + 1:Int32), 27:Int32, (t1.c3 + 1:Int32), 28:Int32, (t1.c1 + 2:Int32), 29:Int32, (t1.c2 + 2:Int32), 30:Int32, (t1.c3 + 2:Int32), 31:Int32, (t1.c1 + 1030:Int32), ((((114514:Int32 + t1.c1) + t1.c2) + t1.c3) + 1919810:Int32)) as $expr1] } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [ConstantLookup(t1.c1, 1:Int32, $expr4, 2:Int32, $expr5, 3:Int32, $expr6, 4:Int32, $expr7, 5:Int32, $expr8, 6:Int32, $expr9, 7:Int32, (t1.c1 + 3:Int32), 8:Int32, (t1.c2 + 3:Int32), 9:Int32, (t1.c3 + 3:Int32), 10:Int32, ($expr1 + 1:Int32), 11:Int32, $expr10, 12:Int32, $expr11, 13:Int32, (t1.c1 + t1.c3), 14:Int32, (t1.c2 + t1.c2), 15:Int32, $expr3, 16:Int32, ($expr1 + 2:Int32), 17:Int32, $expr10, 18:Int32, $expr11, 19:Int32, ($expr12 + 1:Int32), 20:Int32, ($expr13 + 1:Int32), 21:Int32, ($expr14 + 1:Int32), 22:Int32, ($expr12 + t1.c1), 23:Int32, ($expr13 + t1.c2), 24:Int32, ($expr14 + t1.c3), 25:Int32, $expr4, 26:Int32, $expr5, 27:Int32, $expr6, 28:Int32, $expr7, 29:Int32, $expr8, 30:Int32, $expr9, 31:Int32, (t1.c1 + 1030:Int32), ((((114514:Int32 + t1.c1) + t1.c2) + t1.c3) + 1919810:Int32)) as $expr15] } @@ -215,7 +215,7 @@ logical_plan: |- LogicalProject ├─exprs: ConstantLookup(t1.c1, 1:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 2:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 3:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 4:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 5:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 6:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 7:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 8:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 9:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 10:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 11:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 12:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 13:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 14:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 15:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 16:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 17:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 18:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 19:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 20:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 21:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 22:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 23:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 24:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 25:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 26:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 27:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 28:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 29:Int32, RegexpReplace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 30:Int32, RegexpReplace('💗💗💗💗💗亲爱的😭baz这不是爱情❤️‍🔥':Varchar, 'baz(...)':Varchar, '这是🥵':Varchar, 'ic':Varchar), 31:Int32, '😡😡😡这是爱情😭😭😭':Varchar, '❤️‍🔥❤️‍🔥❤️‍🔥❤️‍🔥❤️‍🔥❤️‍🔥':Varchar) as $expr1 - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [ConstantLookup(t1.c1, 1:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 2:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 3:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 4:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 5:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 6:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 7:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 8:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 9:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 10:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 11:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 12:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 13:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 14:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 15:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 16:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 17:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 18:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 19:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 20:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 21:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 22:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 23:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 24:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 25:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 26:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 27:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 28:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 29:Int32, '💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 30:Int32, '💗💗💗💗💗亲爱的😭这是🥵爱情❤️‍🔥':Varchar, 31:Int32, '😡😡😡这是爱情😭😭😭':Varchar, '❤️‍🔥❤️‍🔥❤️‍🔥❤️‍🔥❤️‍🔥❤️‍🔥':Varchar) as $expr1] } @@ -245,7 +245,7 @@ from t1; logical_plan: |- LogicalProject { exprs: [114514:Int32] } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [114514:Int32] } @@ -316,7 +316,7 @@ from t1; logical_plan: |- LogicalProject { exprs: [ConstantLookup(t1.c1, 1:Int32, 'one':Varchar, 2:Int32, 'two':Varchar, 3:Int32, 'three':Varchar, 4:Int32, 'four':Varchar, 5:Int32, 'five':Varchar, 6:Int32, 'six':Varchar, 7:Int32, 'seven':Varchar, 8:Int32, 'eight':Varchar, 9:Int32, 'nine':Varchar, 10:Int32, 'ten':Varchar, 11:Int32, 'eleven':Varchar, 12:Int32, 'twelve':Varchar, 13:Int32, 'thirteen':Varchar, 14:Int32, 'fourteen':Varchar, 15:Int32, 'fifteen':Varchar, 16:Int32, 'sixteen':Varchar, 17:Int32, 'seventeen':Varchar, 18:Int32, 'eighteen':Varchar, 19:Int32, 'nineteen':Varchar, 20:Int32, 'twenty':Varchar, 21:Int32, 'twenty-one':Varchar, 22:Int32, 'twenty-two':Varchar, 23:Int32, 'twenty-three':Varchar, 24:Int32, 'twenty-four':Varchar, 25:Int32, 'twenty-five':Varchar, 26:Int32, 'twenty-six':Varchar, 27:Int32, 'twenty-seven':Varchar, 28:Int32, 'twenty-eight':Varchar, 29:Int32, 'twenty-nine':Varchar, 30:Int32, 'thirty':Varchar, 31:Int32, 'thirty-one':Varchar, 'other':Varchar) as $expr1] } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [ConstantLookup(t1.c1, 1:Int32, 'one':Varchar, 2:Int32, 'two':Varchar, 3:Int32, 'three':Varchar, 4:Int32, 'four':Varchar, 5:Int32, 'five':Varchar, 6:Int32, 'six':Varchar, 7:Int32, 'seven':Varchar, 8:Int32, 'eight':Varchar, 9:Int32, 'nine':Varchar, 10:Int32, 'ten':Varchar, 11:Int32, 'eleven':Varchar, 12:Int32, 'twelve':Varchar, 13:Int32, 'thirteen':Varchar, 14:Int32, 'fourteen':Varchar, 15:Int32, 'fifteen':Varchar, 16:Int32, 'sixteen':Varchar, 17:Int32, 'seventeen':Varchar, 18:Int32, 'eighteen':Varchar, 19:Int32, 'nineteen':Varchar, 20:Int32, 'twenty':Varchar, 21:Int32, 'twenty-one':Varchar, 22:Int32, 'twenty-two':Varchar, 23:Int32, 'twenty-three':Varchar, 24:Int32, 'twenty-four':Varchar, 25:Int32, 'twenty-five':Varchar, 26:Int32, 'twenty-six':Varchar, 27:Int32, 'twenty-seven':Varchar, 28:Int32, 'twenty-eight':Varchar, 29:Int32, 'twenty-nine':Varchar, 30:Int32, 'thirty':Varchar, 31:Int32, 'thirty-one':Varchar, 'other':Varchar) as $expr1] } diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index ce98b8bea75c9..c80253fd16ef3 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -10,7 +10,7 @@ └─LogicalAgg { group_key: [order_line.ol_number], aggs: [sum(order_line.ol_quantity), sum(order_line.ol_amount), count(order_line.ol_quantity), count(order_line.ol_amount), count] } └─LogicalProject { exprs: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount] } └─LogicalFilter { predicate: (order_line.ol_delivery_d > '2007-01-02 00:00:00':Timestamp) } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [order_line.ol_number, sum(order_line.ol_quantity), sum(order_line.ol_amount), (sum(order_line.ol_quantity)::Decimal / count(order_line.ol_quantity)::Decimal) as $expr1, (sum(order_line.ol_amount) / count(order_line.ol_amount)::Decimal) as $expr2, count] } └─LogicalAgg { group_key: [order_line.ol_number], aggs: [sum(order_line.ol_quantity), sum(order_line.ol_amount), count(order_line.ol_quantity), count(order_line.ol_amount), count] } @@ -47,11 +47,11 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ order_line_ol_number, sum(order_line_ol_quantity), sum(order_line_ol_amount), count(order_line_ol_quantity), count(order_line_ol_amount), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ order_line_ol_number, sum(order_line_ol_quantity), sum(order_line_ol_amount), count(order_line_ol_quantity), count(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ ol_number, sum_qty, sum_amount, avg_qty, avg_amount, count_order ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ ol_number, sum_qty, sum_amount, avg_qty, avg_amount, count_order, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: ch_q2 before: @@ -65,11 +65,11 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } - │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - │ └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ │ │ ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + │ └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } └─LogicalProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } └─LogicalAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity)] } └─LogicalProject { exprs: [stock.s_i_id, stock.s_quantity] } @@ -77,10 +77,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment] } ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [item.i_id, item.i_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey] } @@ -277,93 +277,93 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ item_i_id, item_i_name, $expr2, stock_s_i_id, stock_s_w_id, min(stock_s_quantity) ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 0 { columns: [ item_i_id, item_i_name, $expr2, stock_s_i_id, stock_s_w_id, min(stock_s_quantity), _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, stock_s_i_id, stock_s_w_id, min(stock_s_quantity), _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, stock_s_i_id, stock_s_w_id, min(stock_s_quantity), _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 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 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, region_r_regionkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $0 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 3 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ stock_s_i_id, min(stock_s_quantity) ], primary key: [ $0 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4 { columns: [ stock_s_i_id, min(stock_s_quantity), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 5 { columns: [ stock_s_i_id, min(stock_s_quantity), stock_s_i_id_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 5 { columns: [ stock_s_i_id, min(stock_s_quantity), stock_s_i_id_0, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 6 { columns: [ item_i_id, item_i_name, stock_s_i_id, stock_s_w_id, stock_s_quantity ], primary key: [ $0 ASC, $4 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 6 { columns: [ item_i_id, item_i_name, stock_s_i_id, stock_s_w_id, stock_s_quantity, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 7 { columns: [ item_i_id, stock_s_quantity, stock_s_i_id, stock_s_w_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 7 { columns: [ item_i_id, stock_s_quantity, stock_s_i_id, stock_s_w_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 8 { columns: [ stock_s_i_id, stock_s_quantity, region_r_regionkey, supplier_s_suppkey, supplier_s_nationkey, stock_s_w_id ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ stock_s_i_id, stock_s_quantity, region_r_regionkey, supplier_s_suppkey, supplier_s_nationkey, stock_s_w_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ stock_s_i_id, min(stock_s_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ stock_s_i_id, min(stock_s_quantity), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ stock_s_i_id, stock_s_quantity, $expr1, stock_s_w_id ], primary key: [ $2 ASC, $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 12 { columns: [ stock_s_i_id, stock_s_quantity, $expr1, stock_s_w_id, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 13 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 13 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ supplier_s_suppkey, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 16 { columns: [ supplier_s_suppkey, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 17 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 18 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], 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: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 19 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 20 { 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: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ nation_n_nationkey, nation_n_regionkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ nation_n_nationkey, nation_n_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], 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 22 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 23 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 24 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 24 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 25 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 25 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 26 { columns: [ item_i_id, item_i_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 26 { columns: [ item_i_id, item_i_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 27 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 28 { columns: [ stock_s_i_id, stock_s_w_id, stock_s_quantity ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 28 { columns: [ stock_s_i_id, stock_s_w_id, stock_s_quantity, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 29 { columns: [ stock_s_i_id, stock_s_w_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ stock_s_i_id, stock_s_w_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 30 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 31 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 32 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 32 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $6 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + Table 34 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $6 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6 ], read pk prefix len hint: 1 } - Table 35 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 35 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 36 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 36 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 37 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, supplier_s_phone, supplier_s_comment ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 37 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, supplier_s_phone, supplier_s_comment, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 38 { 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 38 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 39 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 39 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 40 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 40 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 41 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 41 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 42 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 42 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 8, 9, 10, 11, 12, 13 ], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 8, 9, 10, 11, 12, 13 ], read pk prefix len hint: 9 } - id: ch_q3 before: @@ -377,10 +377,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } - │ │ └─LogicalScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id] } - │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + │ │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } + │ │ └─LogicalScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id, new_order._rw_timestamp] } + │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } └─LogicalAgg { group_key: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, orders.o_entry_d], aggs: [sum(order_line.ol_amount)] } @@ -479,41 +479,41 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ orders_o_entry_d, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 0 { columns: [ orders_o_entry_d, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, sum(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 1 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, new_order_no_o_id, new_order_no_d_id, new_order_no_w_id ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $5 ASC, $4 ASC, $3 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3, 4, 5 ], read pk prefix len hint: 11 } + Table 1 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, new_order_no_o_id, new_order_no_d_id, new_order_no_w_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $5 ASC, $4 ASC, $3 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3, 4, 5 ], read pk prefix len hint: 11 } - Table 2 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, new_order_no_w_id, new_order_no_d_id, new_order_no_o_id, customer_c_d_id_0, customer_c_w_id_0, new_order_no_o_id_0, new_order_no_d_id_0, new_order_no_w_id_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 11 ], distribution key: [ 5, 4, 3 ], read pk prefix len hint: 11 } + Table 2 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, new_order_no_w_id, new_order_no_d_id, new_order_no_o_id, customer_c_d_id_0, customer_c_w_id_0, new_order_no_o_id_0, new_order_no_d_id_0, new_order_no_w_id_0, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 11 ], distribution key: [ 5, 4, 3 ], read pk prefix len hint: 11 } - Table 3 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $5 ASC, $6 ASC, $7 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 11 } + Table 3 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $5 ASC, $6 ASC, $7 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 11 } - Table 4 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_w_id_0, orders_o_d_id_0, orders_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC ], value indices: [ 12 ], distribution key: [ 5, 4, 3 ], read pk prefix len hint: 11 } + Table 4 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_w_id_0, orders_o_d_id_0, orders_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC ], value indices: [ 12 ], distribution key: [ 5, 4, 3 ], read pk prefix len hint: 11 } - Table 5 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } + Table 5 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } - Table 6 { columns: [ customer_c_d_id, customer_c_w_id, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 6 { columns: [ customer_c_d_id, customer_c_w_id, customer_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 7 { columns: [ new_order_no_o_id, new_order_no_d_id, new_order_no_w_id ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } + Table 7 { columns: [ new_order_no_o_id, new_order_no_d_id, new_order_no_w_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } - Table 8 { columns: [ new_order_no_d_id, new_order_no_w_id, new_order_no_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 8 { columns: [ new_order_no_d_id, new_order_no_w_id, new_order_no_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 9 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 10 { columns: [ vnode, no_w_id, no_d_id, no_o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, no_w_id, no_d_id, no_o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 12 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 12 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 13 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 13 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 14 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 14 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 15 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 15 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 16 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 16 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d ], primary key: [ $3 DESC, $4 ASC, $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4, 0, 2, 1 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d, _rw_timestamp ], primary key: [ $3 DESC, $4 ASC, $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4, 0, 2, 1 ], read pk prefix len hint: 5 } - id: ch_q4 before: @@ -525,10 +525,10 @@ └─LogicalProject { exprs: [orders.o_ol_cnt] } └─LogicalFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } + ├─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } └─LogicalProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } = order_line.ol_o_id) AND (CorrelatedInputRef { index: 2, correlated_id: 1 } = order_line.ol_w_id) AND (CorrelatedInputRef { index: 1, correlated_id: 1 } = order_line.ol_d_id) AND (order_line.ol_delivery_d >= CorrelatedInputRef { index: 4, correlated_id: 1 }) } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [orders.o_ol_cnt], aggs: [count] } └─LogicalJoin { type: LeftSemi, on: (orders.o_id = order_line.ol_o_id) AND (orders.o_w_id = order_line.ol_w_id) AND (orders.o_d_id = order_line.ol_d_id) AND (order_line.ol_delivery_d >= orders.o_entry_d), output: [orders.o_ol_cnt] } @@ -585,21 +585,21 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ orders_o_ol_cnt, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ orders_o_ol_cnt, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_ol_cnt ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 2, 1 ], read pk prefix len hint: 3 } + Table 1 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_ol_cnt, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 2, 1 ], read pk prefix len hint: 3 } - Table 2 { columns: [ orders_o_id, orders_o_w_id, orders_o_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ orders_o_id, orders_o_w_id, orders_o_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_delivery_d, order_line_ol_number ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_delivery_d, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 4 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 5 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ o_ol_cnt, order_count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_ol_cnt, order_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: ch_q5 before: @@ -616,13 +616,13 @@ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } - │ │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ │ │ │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } + │ │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [nation.n_name], aggs: [sum(order_line.ol_amount)] } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [nation.n_name, order_line.ol_amount] } @@ -781,71 +781,71 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ nation_n_name, sum(order_line_ol_amount), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ nation_n_name, sum(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ order_line_ol_amount, $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_id, order_line_ol_number, order_line_ol_i_id, orders_o_w_id, orders_o_d_id ], primary key: [ $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } + Table 1 { columns: [ order_line_ol_amount, $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_id, order_line_ol_number, order_line_ol_i_id, orders_o_w_id, orders_o_d_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_id, order_line_ol_number, order_line_ol_i_id, orders_o_w_id, orders_o_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 2 { columns: [ $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_id, order_line_ol_number, order_line_ol_i_id, orders_o_w_id, orders_o_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 5 { columns: [ customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_id, orders_o_d_id, orders_o_w_id, customer_c_id ], primary key: [ $3 ASC, $5 ASC, $4 ASC, $0 ASC, $1 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 0, 1, 5 ], read pk prefix len hint: 6 } + Table 5 { columns: [ customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_id, orders_o_d_id, orders_o_w_id, customer_c_id, _rw_timestamp ], primary key: [ $3 ASC, $5 ASC, $4 ASC, $0 ASC, $1 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 0, 1, 5 ], read pk prefix len hint: 6 } - Table 6 { columns: [ orders_o_id, orders_o_w_id, orders_o_d_id, customer_c_d_id, customer_c_w_id, orders_o_w_id_0, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 0, 3, 4, 5 ], read pk prefix len hint: 6 } + Table 6 { columns: [ orders_o_id, orders_o_w_id, orders_o_d_id, customer_c_d_id, customer_c_w_id, orders_o_w_id_0, customer_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 0, 3, 4, 5 ], read pk prefix len hint: 6 } - Table 7 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, stock_s_i_id, stock_s_w_id, order_line_ol_number, order_line_ol_i_id ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $1 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 5 ], read pk prefix len hint: 6 } + Table 7 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, stock_s_i_id, stock_s_w_id, order_line_ol_number, order_line_ol_i_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $1 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 5 ], read pk prefix len hint: 6 } - Table 8 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_d_id_0, order_line_ol_w_id_0, stock_s_w_id, order_line_ol_number, order_line_ol_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0, 3, 4, 5 ], read pk prefix len hint: 6 } + Table 8 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_d_id_0, order_line_ol_w_id_0, stock_s_w_id, order_line_ol_number, order_line_ol_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0, 3, 4, 5 ], read pk prefix len hint: 6 } - Table 9 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_state ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } + Table 9 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_state, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } - Table 10 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 10 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } + Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, _rw_timestamp ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } - Table 12 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 12 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 13 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 14 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 15 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $2 ASC, $3 ASC, $1 ASC, $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3 ], read pk prefix len hint: 2 } + Table 15 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $1 ASC, $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3 ], read pk prefix len hint: 2 } - Table 16 { columns: [ order_line_ol_w_id, order_line_ol_i_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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 16 { columns: [ order_line_ol_w_id, order_line_ol_i_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 17 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 17 { columns: [ stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - Table 18 { columns: [ stock_s_w_id, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 18 { columns: [ stock_s_w_id, stock_s_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 19 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 20 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 21 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 22 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], 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, nation_n_name, nation_n_regionkey, nation_n_nationkey ], primary key: [ $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 23 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, nation_n_regionkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 24 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 25 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - 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 26 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], 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 27 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 28 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 28 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 29 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 30 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 31 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ n_name, revenue ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: ch_q6 before: @@ -856,7 +856,7 @@ └─LogicalAgg { aggs: [sum(order_line.ol_amount)] } └─LogicalProject { exprs: [order_line.ol_amount] } └─LogicalFilter { predicate: (order_line.ol_delivery_d >= '1997-01-01 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 100000:Int32) } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum(order_line.ol_amount)] } └─LogicalScan { table: order_line, output_columns: [order_line.ol_amount], required_columns: [order_line.ol_amount, order_line.ol_delivery_d, order_line.ol_quantity], predicate: (order_line.ol_delivery_d >= '1997-01-01 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 100000:Int32) } @@ -894,11 +894,11 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum(order_line_ol_amount)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum(order_line_ol_amount)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], 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 } + Table 4294967294 { columns: [ revenue, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: ch_q7 before: @@ -916,13 +916,13 @@ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - │ │ └─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + │ │ └─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [supplier.s_nationkey, $expr3, $expr4, sum(order_line.ol_amount), supplier.s_nationkey] } └─LogicalAgg { group_key: [supplier.s_nationkey, $expr3, $expr4], aggs: [sum(order_line.ol_amount)] } @@ -1087,71 +1087,71 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_nationkey, $expr3, $expr4, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ supplier_s_nationkey, $expr3, $expr4, sum(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 1 { columns: [ order_line_ol_amount, orders_o_entry_d, customer_c_state, nation_n_name, $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, $expr1 ], primary key: [ $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 1 { columns: [ order_line_ol_amount, orders_o_entry_d, customer_c_state, nation_n_name, $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, $expr1, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, nation_n_nationkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_amount, orders_o_entry_d, customer_c_state, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id ], primary key: [ $5 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 5 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_amount, orders_o_entry_d, customer_c_state, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, _rw_timestamp ], primary key: [ $5 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { 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 7 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $3 ASC, $4 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 2, 3, 4, 3, 4 ], read pk prefix len hint: 5 } + Table 9 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $3 ASC, $4 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 2, 3, 4, 3, 4 ], read pk prefix len hint: 5 } - Table 10 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, stock_s_w_id, stock_s_i_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 2, 1, 0, 3, 4 ], read pk prefix len hint: 5 } + Table 10 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, stock_s_w_id, stock_s_i_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 2, 1, 0, 3, 4 ], read pk prefix len hint: 5 } - Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 1, 2, 4, 5 ], read pk prefix len hint: 5 } + Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 1, 2, 4, 5 ], read pk prefix len hint: 5 } - Table 12 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1, 0, 3, 4 ], read pk prefix len hint: 5 } + Table 12 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, orders_o_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1, 0, 3, 4 ], read pk prefix len hint: 5 } - Table 13 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 13 { columns: [ stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 14 { columns: [ stock_s_w_id, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 14 { columns: [ stock_s_w_id, stock_s_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - Table 15 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 4 ], read pk prefix len hint: 2 } + Table 15 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 4 ], read pk prefix len hint: 2 } - Table 16 { columns: [ order_line_ol_supply_w_id, order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 16 { columns: [ order_line_ol_supply_w_id, order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - Table 17 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 17 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 18 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 18 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 19 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } + Table 19 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, _rw_timestamp ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } - Table 20 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 20 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 21 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_state ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } + Table 21 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_state, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } - Table 22 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 22 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 23 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 23 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 24 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 24 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 25 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 25 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - 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 26 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], 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 27 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 28 { 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 28 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 29 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 30 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 31 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ supp_nation, cust_nation, l_year, revenue, supplier.s_nationkey ], primary key: [ $4 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4, 1, 2 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue, supplier.s_nationkey, _rw_timestamp ], primary key: [ $4 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4, 1, 2 ], read pk prefix len hint: 3 } - id: ch_q8 before: @@ -1170,15 +1170,15 @@ │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ │ │ │ ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } - │ │ │ │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - │ │ │ └─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } - │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ │ │ │ │ │ ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } + │ │ │ │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + │ │ │ └─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } + │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [$expr3, (sum($expr4) / sum(order_line.ol_amount)) as $expr5] } └─LogicalAgg { group_key: [$expr3], aggs: [sum($expr4), sum(order_line.ol_amount)] } @@ -1388,96 +1388,96 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ $expr3, sum($expr4), sum(order_line_ol_amount), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr3, sum($expr4), sum(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ orders_o_entry_d, order_line_ol_amount, nation_n_name, $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, supplier_s_nationkey, $expr1 ] + ├── columns: [ orders_o_entry_d, order_line_ol_amount, nation_n_name, $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, supplier_s_nationkey, $expr1, _rw_timestamp ] ├── primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 3 ] └── read pk prefix len hint: 1 - Table 2 { columns: [ $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, supplier_s_nationkey, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ], value indices: [ 13 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, supplier_s_nationkey, $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ], value indices: [ 13 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ nation_n_nationkey, nation_n_regionkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ nation_n_nationkey, nation_n_regionkey, region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ nation_n_nationkey, nation_n_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ nation_n_nationkey, nation_n_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ orders_o_entry_d, customer_c_state, order_line_ol_amount, $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id ], primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 5 { columns: [ orders_o_entry_d, customer_c_state, order_line_ol_amount, $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, _rw_timestamp ], primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 11 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 11 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 5 } + Table 9 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 5 } - Table 10 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } + Table 10 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, orders_o_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } - Table 11 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, item_i_id, order_line_ol_number ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $3 ASC, $4 ASC, $6 ASC, $1 ASC, $7 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } + Table 11 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, item_i_id, order_line_ol_number, _rw_timestamp ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $3 ASC, $4 ASC, $6 ASC, $1 ASC, $7 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } - Table 12 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } + Table 12 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } - Table 13 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } + Table 13 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, _rw_timestamp ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } - Table 14 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 14 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 15 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_state ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } + Table 15 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_state, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } - Table 16 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 16 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 17 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 17 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 18 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 18 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 19 { columns: [ item_i_id, stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 3 } + Table 19 { columns: [ item_i_id, stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 3 } - Table 20 { columns: [ stock_s_i_id, stock_s_w_id, item_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 20 { columns: [ stock_s_i_id, stock_s_w_id, item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 21 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $3 ASC, $4 ASC, $3 ASC, $2 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 4 ], read pk prefix len hint: 3 } + Table 21 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $3 ASC, $4 ASC, $3 ASC, $2 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 4 ], read pk prefix len hint: 3 } - Table 22 { columns: [ order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_i_id_0, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 22 { columns: [ order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_i_id_0, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 23 { columns: [ item_i_id ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ item_i_id, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 25 { columns: [ stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ stock_s_i_id, stock_s_w_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 26 { columns: [ stock_s_i_id, stock_s_w_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 27 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 28 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 28 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 29 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 29 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 30 { 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 30 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 31 { 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 31 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 32 { 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 32 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 34 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 35 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 35 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 36 { 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 36 { columns: [ nation_n_nationkey, nation_n_regionkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 37 { 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 37 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 38 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 38 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 39 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 39 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 40 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 40 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 41 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 41 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], 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_year, mkt_share ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ l_year, mkt_share, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: ch_q9 before: @@ -1493,12 +1493,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } - │ │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ │ │ ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [nation.n_name, $expr2], aggs: [sum(order_line.ol_amount)] } └─LogicalProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_entry_d) as $expr2, order_line.ol_amount] } @@ -1639,61 +1639,61 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ nation_n_name, $expr2, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ nation_n_name, $expr2, sum(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 1 { columns: [ orders_o_entry_d, order_line_ol_amount, $expr1, item_i_id, stock_s_w_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, stock_s_i_id ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 1 { columns: [ orders_o_entry_d, order_line_ol_amount, $expr1, item_i_id, stock_s_w_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, stock_s_i_id, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr1, item_i_id, stock_s_w_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr1, item_i_id, stock_s_w_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, stock_s_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ item_i_id, stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 3 } + Table 5 { columns: [ item_i_id, stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 3 } - Table 6 { columns: [ stock_s_i_id, stock_s_w_id, item_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 6 { columns: [ stock_s_i_id, stock_s_w_id, item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 7 { columns: [ orders_o_entry_d, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $1 ASC, $2 ASC, $1 ASC, $4 ASC, $5 ASC, $6 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 2 ], read pk prefix len hint: 3 } + Table 7 { columns: [ orders_o_entry_d, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $1 ASC, $4 ASC, $5 ASC, $6 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 2 ], read pk prefix len hint: 3 } - Table 8 { columns: [ order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_i_id_0, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 8 { columns: [ order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_i_id_0, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 9 { columns: [ item_i_id ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ item_i_id, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ stock_s_i_id, stock_s_w_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ stock_s_i_id, stock_s_w_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 15 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 15 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 16 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 16 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 17 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 17 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 18 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 18 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 19 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 20 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 21 { 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 21 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 22 { 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 22 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { 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 23 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 25 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 26 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 26 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ n_name, l_year, sum_profit ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ n_name, l_year, sum_profit, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: ch_q10 before: @@ -1707,10 +1707,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } - │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } + │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } └─LogicalAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount)] } @@ -1813,41 +1813,41 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ customer_c_id, customer_c_last, customer_c_city, customer_c_phone, nation_n_name, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + Table 0 { columns: [ customer_c_id, customer_c_last, customer_c_city, customer_c_phone, nation_n_name, sum(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } - Table 1 { columns: [ order_line_ol_amount, customer_c_id, customer_c_last, customer_c_city, customer_c_phone, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 1 { columns: [ order_line_ol_amount, customer_c_id, customer_c_last, customer_c_city, customer_c_phone, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, customer_c_id, _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 2 { columns: [ $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, customer_c_id, _degree, _rw_timestamp ], 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 3 { 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 3 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, order_line_ol_amount, order_line_ol_number ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $1 ASC, $2 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 2, 0, 1, 2 ], read pk prefix len hint: 5 } + Table 5 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $1 ASC, $2 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 2, 0, 1, 2 ], read pk prefix len hint: 5 } - Table 6 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 3, 4, 2, 1, 0 ], read pk prefix len hint: 5 } + Table 6 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 3, 4, 2, 1, 0 ], read pk prefix len hint: 5 } - Table 7 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_last, customer_c_city, customer_c_state, customer_c_phone, orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d ], primary key: [ $9 ASC, $8 ASC, $7 ASC, $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 2, 7, 8, 9 ], read pk prefix len hint: 5 } + Table 7 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_last, customer_c_city, customer_c_state, customer_c_phone, orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, _rw_timestamp ], primary key: [ $9 ASC, $8 ASC, $7 ASC, $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 2, 7, 8, 9 ], read pk prefix len hint: 5 } - Table 8 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 3, 4, 2, 1, 0 ], read pk prefix len hint: 5 } + Table 8 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, customer_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 3, 4, 2, 1, 0 ], read pk prefix len hint: 5 } - Table 9 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 10 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_last, customer_c_city, customer_c_state, customer_c_phone ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } + Table 10 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_last, customer_c_city, customer_c_state, customer_c_phone, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } - Table 11 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 11 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 12 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } + Table 12 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, _rw_timestamp ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } - Table 13 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 13 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 14 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 15 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 15 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 16 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 16 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ c_id, c_last, revenue, c_city, c_phone, n_name ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 3, 4, 5 ], read pk prefix len hint: 6 } + Table 4294967294 { columns: [ c_id, c_last, revenue, c_city, c_phone, n_name, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 3, 4, 5 ], read pk prefix len hint: 6 } - id: ch_q11 before: @@ -1862,18 +1862,18 @@ │ └─LogicalFilter { predicate: (((stock.s_w_id * stock.s_i_id) % 10000:Int32) = supplier.s_suppkey) AND (supplier.s_nationkey = nation.n_nationkey) AND (nation.n_name = 'CHINA':Varchar) } │ └─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } └─LogicalProject { exprs: [(sum(stock.s_order_cnt)::Decimal * 0.005:Decimal) as $expr1] } └─LogicalAgg { aggs: [sum(stock.s_order_cnt)] } └─LogicalProject { exprs: [stock.s_order_cnt] } └─LogicalFilter { predicate: (((stock.s_w_id * stock.s_i_id) % 10000:Int32) = supplier.s_suppkey) AND (supplier.s_nationkey = nation.n_nationkey) AND (nation.n_name = 'CHINA':Varchar) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: ($expr2 > $expr4), output: [stock.s_i_id, sum(stock.s_order_cnt)] } ├─LogicalProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } @@ -2016,37 +2016,37 @@ StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ stock_s_i_id, sum(stock_s_order_cnt), $expr2 ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ stock_s_i_id, sum(stock_s_order_cnt), $expr2, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr3 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr3, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ stock_s_i_id, sum(stock_s_order_cnt), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ stock_s_i_id, sum(stock_s_order_cnt), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ stock_s_i_id, stock_s_order_cnt, $expr1, stock_s_w_id ], primary key: [ $2 ASC, $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 5 { columns: [ stock_s_i_id, stock_s_order_cnt, $expr1, stock_s_w_id, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { 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 7 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 8 { 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 8 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 14 { columns: [ sum(sum(stock_s_order_cnt)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 14 { columns: [ sum(sum(stock_s_order_cnt)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ s_i_id, ordercount ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_i_id, ordercount, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: ch_q12 before: @@ -2058,8 +2058,8 @@ └─LogicalProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2] } └─LogicalFilter { predicate: (order_line.ol_w_id = orders.o_w_id) AND (order_line.ol_d_id = orders.o_d_id) AND (order_line.ol_o_id = orders.o_id) AND (orders.o_entry_d <= order_line.ol_delivery_d) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + ├─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [orders.o_ol_cnt], aggs: [sum($expr1), sum($expr2)] } └─LogicalProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2] } @@ -2119,21 +2119,21 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ orders_o_ol_cnt, sum($expr1), sum($expr2), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ orders_o_ol_cnt, sum($expr1), sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_carrier_id, orders_o_ol_cnt ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 1 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_carrier_id, orders_o_ol_cnt, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 2 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 2 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 3 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, order_line_ol_number ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, order_line_ol_number, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 4 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 5 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ o_ol_cnt, high_line_count, low_line_count ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_ol_cnt, high_line_count, low_line_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: ch_q13 before: @@ -2147,8 +2147,8 @@ └─LogicalAgg { group_key: [customer.c_id], aggs: [count(orders.o_id)] } └─LogicalProject { exprs: [customer.c_id, orders.o_id] } └─LogicalJoin { type: LeftOuter, on: (customer.c_w_id = orders.o_w_id) AND (customer.c_d_id = orders.o_d_id) AND (customer.c_id = orders.o_c_id) AND (orders.o_carrier_id > 8:Int32), output: all } - ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } - └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } + ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } + └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [count(orders.o_id)], aggs: [count] } └─LogicalAgg { group_key: [customer.c_id], aggs: [count(orders.o_id)] } @@ -2215,23 +2215,23 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ count(orders_o_id), count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ count(orders_o_id), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ customer_c_id, count(orders_o_id), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ customer_c_id, count(orders_o_id), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } + Table 2 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } - Table 3 { columns: [ customer_c_w_id, customer_c_d_id, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 1, 0, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ customer_c_w_id, customer_c_d_id, customer_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 1, 0, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } + Table 4 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } - Table 5 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_c_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 1, 0, 2 ], read pk prefix len hint: 3 } + Table 5 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_c_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 1, 0, 2 ], read pk prefix len hint: 3 } - Table 6 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 7 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], 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: 2 } + Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: ch_q14 before: @@ -2243,8 +2243,8 @@ └─LogicalProject { exprs: [Case(Like(item.i_data, 'PR%':Varchar), order_line.ol_amount, 0:Int32::Decimal) as $expr1, order_line.ol_amount] } └─LogicalFilter { predicate: (order_line.ol_i_id = item.i_id) AND (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-02 00:00:00':Timestamp) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } + ├─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [((100.00:Decimal * sum($expr1)) / (1:Int32::Decimal + sum(order_line.ol_amount))) as $expr2] } └─LogicalAgg { aggs: [sum($expr1), sum(order_line.ol_amount)] } @@ -2307,21 +2307,21 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum($expr1)), sum(sum(order_line_ol_amount)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum($expr1)), sum(sum(order_line_ol_amount)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ order_line_ol_i_id, order_line_ol_amount, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ order_line_ol_i_id, order_line_ol_amount, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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 2 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 3 { columns: [ item_i_id, item_i_data ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ item_i_id, item_i_data, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], 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: [ promo_revenue ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ promo_revenue, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: ch_q15 before: @@ -2332,12 +2332,12 @@ └─LogicalFilter { predicate: (supplier.s_suppkey = revenue1.supplier_no) AND (revenue1.total_revenue = max(revenue1.total_revenue)) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue] } + │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue, revenue1._rw_timestamp] } └─LogicalProject { exprs: [max(revenue1.total_revenue)] } └─LogicalAgg { aggs: [max(revenue1.total_revenue)] } └─LogicalProject { exprs: [revenue1.total_revenue] } - └─LogicalScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue] } + └─LogicalScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue, revenue1._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (revenue1.total_revenue = max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue] } ├─LogicalJoin { type: Inner, on: (supplier.s_suppkey = $expr1), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue] } @@ -2409,41 +2409,41 @@ └── BatchPlanNode Table 0 - ├── columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, revenue1_total_revenue, revenue1_supplier_no ] + ├── columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, revenue1_total_revenue, revenue1_supplier_no, _rw_timestamp ] ├── primary key: [ $4 ASC, $0 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 4 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ revenue1_total_revenue, supplier_s_suppkey, revenue1_supplier_no, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ revenue1_total_revenue, supplier_s_suppkey, revenue1_supplier_no, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ max(max(revenue1_total_revenue)) ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ max(max(revenue1_total_revenue)), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ max(max(revenue1_total_revenue)), _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ max(max(revenue1_total_revenue)), _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ revenue1_total_revenue, $expr1, revenue1_supplier_no ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 6 { columns: [ revenue1_total_revenue, $expr1, revenue1_supplier_no, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 7 { columns: [ $expr1, revenue1_supplier_no, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ $expr1, revenue1_supplier_no, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, supplier_no, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, supplier_no, backfill_finished, row_count, _rw_timestamp ], 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: [ max(revenue1_total_revenue), _vnode ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 10 { columns: [ max(revenue1_total_revenue), _vnode, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 11 { columns: [ max(max(revenue1_total_revenue)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 11 { columns: [ max(max(revenue1_total_revenue)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 12 { columns: [ _vnode, revenue1_total_revenue, revenue1_supplier_no ], 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 12 { columns: [ _vnode, revenue1_total_revenue, revenue1_supplier_no, _rw_timestamp ], 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: [ _vnode, max(revenue1_total_revenue), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ _vnode, max(revenue1_total_revenue), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } - Table 14 { columns: [ vnode, supplier_no, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ vnode, supplier_no, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 3 } - id: ch_q16 before: @@ -2456,11 +2456,11 @@ └─LogicalFilter { predicate: (item.i_id = stock.s_i_id) AND Not(Like(item.i_data, 'zz%':Varchar)) } └─LogicalApply { type: LeftAnti, on: (((stock.s_w_id * stock.s_i_id) % 10000:Int32) = supplier.s_suppkey), correlated_id: 1 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } + │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } └─LogicalProject { exprs: [supplier.s_suppkey] } └─LogicalFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } - └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } + └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count($expr3)] } └─LogicalAgg { group_key: [item.i_name, $expr2, item.i_price, $expr3], aggs: [] } @@ -2546,38 +2546,43 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ item_i_name, $expr2, item_i_price, count(distinct $expr3), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ item_i_name, $expr2, item_i_price, count(distinct $expr3), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 1 { columns: [ item_i_name, $expr2, item_i_price, $expr3, count_for_agg_call_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 4 } + Table 1 + ├── columns: [ item_i_name, $expr2, item_i_price, $expr3, count_for_agg_call_0, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 4 ] + ├── distribution key: [ 0, 1, 2 ] + └── read pk prefix len hint: 4 Table 2 - ├── columns: [ stock_s_i_id, stock_s_w_id, item_i_name, item_i_price, item_i_data, $expr1 ] + ├── columns: [ stock_s_i_id, stock_s_w_id, item_i_name, item_i_price, item_i_data, $expr1, _rw_timestamp ] ├── primary key: [ $5 ASC, $1 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 5 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ stock_s_i_id, stock_s_w_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ stock_s_i_id, stock_s_w_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ item_i_id, item_i_name, item_i_price, item_i_data ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ item_i_id, item_i_name, item_i_price, item_i_data, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 11 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], 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, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ i_name, brand, i_price, 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: 4 } + Table 4294967294 { columns: [ i_name, brand, i_price, supplier_cnt, _rw_timestamp ], 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: 4 } - id: ch_q17 before: @@ -2589,14 +2594,14 @@ └─LogicalProject { exprs: [order_line.ol_amount] } └─LogicalFilter { predicate: (order_line.ol_i_id = item.i_id) AND (order_line.ol_quantity::Decimal < $expr1) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + ├─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } └─LogicalProject { exprs: [item.i_id, (sum(order_line.ol_quantity)::Decimal / count(order_line.ol_quantity)::Decimal) as $expr1] } └─LogicalAgg { group_key: [item.i_id], aggs: [sum(order_line.ol_quantity), count(order_line.ol_quantity)] } └─LogicalProject { exprs: [item.i_id, order_line.ol_quantity] } └─LogicalFilter { predicate: Like(item.i_data, '%b':Varchar) AND (order_line.ol_i_id = item.i_id) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + ├─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(sum(order_line.ol_amount) / 2.0:Decimal) as $expr3] } └─LogicalAgg { aggs: [sum(order_line.ol_amount)] } @@ -2689,33 +2694,33 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum(order_line_ol_amount)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum(order_line_ol_amount)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ order_line_ol_i_id, order_line_ol_amount, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ order_line_ol_i_id, order_line_ol_amount, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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 2 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 3 { columns: [ item_i_id, $expr2 ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ item_i_id, $expr2, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ item_i_id, sum(order_line_ol_quantity), count(order_line_ol_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ item_i_id, sum(order_line_ol_quantity), count(order_line_ol_quantity), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ item_i_id ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ item_i_id, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ order_line_ol_i_id, order_line_ol_quantity, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ order_line_ol_i_id, order_line_ol_quantity, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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 10 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 11 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], 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, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ avg_yearly ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ avg_yearly, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: ch_q18 before: @@ -2730,9 +2735,9 @@ └─LogicalFilter { predicate: (customer.c_id = orders.o_c_id) AND (customer.c_w_id = orders.o_w_id) AND (customer.c_d_id = orders.o_d_id) AND (order_line.ol_w_id = orders.o_w_id) AND (order_line.ol_d_id = orders.o_d_id) AND (order_line.ol_o_id = orders.o_id) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } - │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } + │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount)] } └─LogicalFilter { predicate: (sum(order_line.ol_amount) > 200:Int32::Decimal) } @@ -2807,31 +2812,31 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ customer_c_id, customer_c_last, orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_ol_cnt, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7, 8 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 7 } + Table 0 { columns: [ customer_c_id, customer_c_last, orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_ol_cnt, sum(order_line_ol_amount), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7, 8 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 7 } - Table 1 { columns: [ customer_c_id, customer_c_last, orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_ol_cnt, customer_c_w_id, customer_c_d_id ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $7 ASC, $8 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 3 } + Table 1 { columns: [ customer_c_id, customer_c_last, orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, orders_o_ol_cnt, customer_c_w_id, customer_c_d_id, _rw_timestamp ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $7 ASC, $8 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 3 } - Table 2 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 2 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 3 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, order_line_ol_number, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 4 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 5 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_last ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } + Table 5 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_last, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } - Table 6 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 6 { columns: [ customer_c_id, customer_c_w_id, customer_c_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 7 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, orders_o_ol_cnt ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } + Table 7 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d, orders_o_ol_cnt, _rw_timestamp ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } - Table 8 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } + Table 8 { columns: [ orders_o_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1, 0 ], read pk prefix len hint: 3 } - Table 9 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, c_w_id, c_d_id, c_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 10 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 11 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id ], primary key: [ $5 DESC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6, 7, 8 ], read pk prefix len hint: 8 } + Table 4294967294 { columns: [ c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id, _rw_timestamp ], primary key: [ $5 DESC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6, 7, 8 ], read pk prefix len hint: 8 } - id: ch_q19 before: @@ -2843,8 +2848,8 @@ └─LogicalProject { exprs: [order_line.ol_amount] } └─LogicalFilter { predicate: (order_line.ol_i_id = item.i_id) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 10:Int32) AND (item.i_price >= 1:Int32::Decimal) AND (item.i_price <= 400000:Int32::Decimal) AND (((Like(item.i_data, '%a':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 3:Int32)) OR (Like(item.i_data, '%b':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 4:Int32))) OR (Like(item.i_data, '%c':Varchar) AND In(order_line.ol_w_id, 1:Int32, 5:Int32, 3:Int32))) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } + ├─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum(order_line.ol_amount)] } └─LogicalJoin { type: Inner, on: (order_line.ol_i_id = item.i_id) AND (((Like(item.i_data, '%a':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 3:Int32)) OR (Like(item.i_data, '%b':Varchar) AND In(order_line.ol_w_id, 1:Int32, 2:Int32, 4:Int32))) OR (Like(item.i_data, '%c':Varchar) AND In(order_line.ol_w_id, 1:Int32, 5:Int32, 3:Int32))), output: [order_line.ol_amount] } @@ -2908,21 +2913,21 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum(order_line_ol_amount)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum(order_line_ol_amount)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ order_line_ol_w_id, order_line_ol_i_id, order_line_ol_amount, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $1 ASC, $0 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 1 { columns: [ order_line_ol_w_id, order_line_ol_i_id, order_line_ol_amount, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 2 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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 2 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 3 { columns: [ item_i_id, item_i_data ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ item_i_id, item_i_data, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], 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 } + Table 4294967294 { columns: [ revenue, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: ch_q20 before: @@ -2933,8 +2938,8 @@ └─LogicalFilter { predicate: (supplier.s_nationkey = nation.n_nationkey) AND (nation.n_name = 'CHINA':Varchar) } └─LogicalApply { type: LeftSemi, on: (supplier.s_suppkey = $expr1), correlated_id: 1 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } └─LogicalProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32) as $expr1] } └─LogicalFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } └─LogicalAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity)] } @@ -2942,11 +2947,11 @@ └─LogicalFilter { predicate: (order_line.ol_i_id = stock.s_i_id) AND (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } └─LogicalApply { type: LeftSemi, on: (stock.s_i_id = item.i_id), correlated_id: 2 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + │ ├─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } └─LogicalProject { exprs: [item.i_id] } └─LogicalFilter { predicate: Like(item.i_data, 'co%':Varchar) } - └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data] } + └─LogicalScan { table: item, columns: [item.i_id, item.i_im_id, item.i_name, item.i_price, item.i_data, item._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (supplier.s_suppkey = $expr1), output: [supplier.s_name, supplier.s_address] } ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address] } @@ -3073,51 +3078,51 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr1, stock_s_i_id, stock_s_w_id, stock_s_quantity ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr1, stock_s_i_id, stock_s_w_id, stock_s_quantity, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr1, stock_s_i_id, stock_s_w_id, stock_s_quantity, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr1, stock_s_i_id, stock_s_w_id, stock_s_quantity, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 5 { 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 5 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ stock_s_i_id, stock_s_w_id, stock_s_quantity, sum(order_line_ol_quantity), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 10 { columns: [ stock_s_i_id, stock_s_w_id, stock_s_quantity, sum(order_line_ol_quantity), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 11 { columns: [ stock_s_i_id, stock_s_w_id, stock_s_quantity, order_line_ol_quantity, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $0 ASC, $1 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 11 { columns: [ stock_s_i_id, stock_s_w_id, stock_s_quantity, order_line_ol_quantity, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $1 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 12 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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 12 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 13 { columns: [ item_i_id ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 13 { columns: [ item_i_id, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ item_i_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ item_i_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ stock_s_i_id, stock_s_w_id, stock_s_quantity ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ stock_s_i_id, stock_s_w_id, stock_s_quantity, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ stock_s_i_id, stock_s_w_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ stock_s_i_id, stock_s_w_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ order_line_ol_i_id, order_line_ol_quantity, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ order_line_ol_i_id, order_line_ol_quantity, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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: [ order_line_ol_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 19 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 20 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 21 { columns: [ vnode, i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 21 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } - id: ch_q21 before: @@ -3133,14 +3138,14 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } - │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } + │ │ │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } + │ │ └─LogicalScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity, stock.s_dist_01, stock.s_dist_02, stock.s_dist_03, stock.s_dist_04, stock.s_dist_05, stock.s_dist_06, stock.s_dist_07, stock.s_dist_08, stock.s_dist_09, stock.s_dist_10, stock.s_ytd, stock.s_order_cnt, stock.s_remote_cnt, stock.s_data, stock._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } └─LogicalProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } - └─LogicalFilter { predicate: (order_line.ol_o_id = CorrelatedInputRef { index: 7, correlated_id: 1 }) AND (order_line.ol_w_id = CorrelatedInputRef { index: 9, correlated_id: 1 }) AND (order_line.ol_d_id = CorrelatedInputRef { index: 8, correlated_id: 1 }) AND (order_line.ol_delivery_d > CorrelatedInputRef { index: 13, correlated_id: 1 }) } - └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info] } + └─LogicalFilter { predicate: (order_line.ol_o_id = CorrelatedInputRef { index: 8, correlated_id: 1 }) AND (order_line.ol_w_id = CorrelatedInputRef { index: 10, correlated_id: 1 }) AND (order_line.ol_d_id = CorrelatedInputRef { index: 9, correlated_id: 1 }) AND (order_line.ol_delivery_d > CorrelatedInputRef { index: 14, correlated_id: 1 }) } + └─LogicalScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_delivery_d, order_line.ol_quantity, order_line.ol_amount, order_line.ol_dist_info, order_line._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { group_key: [supplier.s_name], aggs: [count] } └─LogicalJoin { type: LeftAnti, on: (order_line.ol_o_id = order_line.ol_o_id) AND (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_delivery_d > order_line.ol_delivery_d), output: [supplier.s_name] } @@ -3280,61 +3285,61 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_name, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ supplier_s_name, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ supplier_s_name, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, stock_s_w_id, stock_s_i_id, order_line_ol_number, $expr1, supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $1 ASC, $3 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC, $10 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 3, 2 ], read pk prefix len hint: 3 } + Table 1 { columns: [ supplier_s_name, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, stock_s_w_id, stock_s_i_id, order_line_ol_number, $expr1, supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $3 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC, $10 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 3, 2 ], read pk prefix len hint: 3 } - Table 2 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, stock_s_w_id, stock_s_i_id, order_line_ol_number, supplier_s_nationkey, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, stock_s_w_id, stock_s_i_id, order_line_ol_number, supplier_s_nationkey, $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_delivery_d, order_line_ol_number ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_delivery_d, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 4 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 5 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_number ], primary key: [ $4 ASC, $5 ASC, $6 ASC, $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 5 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_number, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC, $6 ASC, $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _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 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], 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 7 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0, 1 ], read pk prefix len hint: 3 } + Table 9 { columns: [ stock_s_i_id, stock_s_w_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0, 1 ], read pk prefix len hint: 3 } - Table 10 { columns: [ stock_s_w_id, stock_s_i_id, stock_s_w_id_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 10 { columns: [ stock_s_w_id, stock_s_i_id, stock_s_w_id_0, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 11 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_delivery_d, orders_o_w_id, order_line_ol_number, orders_o_d_id, orders_o_id ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2, 3, 5 ], read pk prefix len hint: 3 } + Table 11 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_delivery_d, orders_o_w_id, order_line_ol_number, orders_o_d_id, orders_o_id, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2, 3, 5 ], read pk prefix len hint: 3 } - Table 12 { columns: [ order_line_ol_w_id, order_line_ol_i_id, orders_o_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 12 { columns: [ order_line_ol_w_id, order_line_ol_i_id, orders_o_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 13 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ vnode, s_w_id, s_i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 14 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_delivery_d, order_line_ol_number ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 14 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_i_id, order_line_ol_delivery_d, order_line_ol_number, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 15 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 2, 1 ], read pk prefix len hint: 3 } + Table 15 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_number, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 2, 1 ], read pk prefix len hint: 3 } - Table 16 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 16 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 17 { columns: [ orders_o_id, orders_o_w_id, orders_o_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 2, 1 ], read pk prefix len hint: 3 } + Table 17 { columns: [ orders_o_id, orders_o_w_id, orders_o_d_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 2, 1 ], read pk prefix len hint: 3 } - Table 18 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 18 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 19 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 20 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 20 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 21 { 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 21 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 22 { columns: [ nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 24 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 25 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 25 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 26 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 26 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, numwait ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: ch_q22 before: @@ -3348,15 +3353,15 @@ └─LogicalFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) AND (customer.c_balance > $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } ├─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } - │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } + │ ├─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } │ └─LogicalProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } │ └─LogicalFilter { predicate: (orders.o_c_id = CorrelatedInputRef { index: 0, correlated_id: 1 }) AND (orders.o_w_id = CorrelatedInputRef { index: 2, correlated_id: 1 }) AND (orders.o_d_id = CorrelatedInputRef { index: 1, correlated_id: 1 }) } - │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local] } + │ └─LogicalScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt, orders.o_all_local, orders._rw_timestamp] } └─LogicalProject { exprs: [(sum(customer.c_balance) / count(customer.c_balance)::Decimal) as $expr1] } └─LogicalAgg { aggs: [sum(customer.c_balance), count(customer.c_balance)] } └─LogicalProject { exprs: [customer.c_balance] } └─LogicalFilter { predicate: (customer.c_balance > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) } - └─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data] } + └─LogicalScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_first, customer.c_middle, customer.c_last, customer.c_street_1, customer.c_street_2, customer.c_city, customer.c_state, customer.c_zip, customer.c_phone, customer.c_since, customer.c_credit, customer.c_credit_lim, customer.c_discount, customer.c_balance, customer.c_ytd_payment, customer.c_payment_cnt, customer.c_delivery_cnt, customer.c_data, customer._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [$expr2, count, sum(customer.c_balance), $expr2] } └─LogicalAgg { group_key: [$expr2], aggs: [count, sum(customer.c_balance)] } diff --git a/src/frontend/planner_test/tests/testdata/output/column_pruning.yaml b/src/frontend/planner_test/tests/testdata/output/column_pruning.yaml index 3daad2d32bd4a..20f496f3f8bbc 100644 --- a/src/frontend/planner_test/tests/testdata/output/column_pruning.yaml +++ b/src/frontend/planner_test/tests/testdata/output/column_pruning.yaml @@ -4,7 +4,7 @@ select v1 from t logical_plan: |- LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: 'LogicalScan { table: t, columns: [t.v1] }' - name: filter sql: | @@ -13,7 +13,7 @@ logical_plan: |- LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v2 > 2:Int32::Float64) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: 'LogicalScan { table: t, output_columns: [t.v1], required_columns: [t.v1, t.v2], predicate: (t.v2 > 2:Int32::Float64) }' - name: join sql: | @@ -23,8 +23,8 @@ logical_plan: |- LogicalProject { exprs: [t1.v1, t2.v1] } └─LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: [t1.v1, t2.v1] } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2] } @@ -38,7 +38,7 @@ └─LogicalAgg { aggs: [count(t.v1)] } └─LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v2 > 2:Int32::Float64) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count(t.v1)] } └─LogicalScan { table: t, output_columns: [t.v1], required_columns: [t.v1, t.v2], predicate: (t.v2 > 2:Int32::Float64) } @@ -50,7 +50,7 @@ LogicalProject { exprs: [t.v3] } └─LogicalTopN { order: [t.v3 ASC, t.v2 ASC], limit: 2, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.v3] } └─LogicalTopN { order: [t.v3 ASC, t.v2 ASC], limit: 2, offset: 0 } @@ -61,7 +61,7 @@ select 1 from t logical_plan: |- LogicalProject { exprs: [1:Int32] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, columns: [] } @@ -72,7 +72,7 @@ logical_plan: |- LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (t.v2 > 1:Int32::Float64) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [1:Int32] } └─LogicalScan { table: t, output_columns: [], required_columns: [t.v2], predicate: (t.v2 > 1:Int32::Float64) } @@ -84,7 +84,7 @@ LogicalProject { exprs: [count(1:Int32)] } └─LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } @@ -98,7 +98,7 @@ └─LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (t.v2 > 1:Int32::Float64) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } @@ -112,8 +112,8 @@ LogicalProject { exprs: [t1.v1, t2.v1] } └─LogicalFilter { predicate: (t1.v3 < 1:Int32) } └─LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (t1.v2 = t2.v2), output: [t1.v1, t2.v1] } ├─LogicalScan { table: t1, output_columns: [t1.v1, t1.v2], required_columns: [t1.v1, t1.v2, t1.v3], predicate: (t1.v3 < 1:Int32) } @@ -128,7 +128,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($4)] } └─LogicalProjectSet { select_list: [$0, $1, $2, Unnest($4)] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.arr, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.arr, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($3)] } └─LogicalFilter { predicate: (Unnest($3) = 30:Int32) } @@ -146,8 +146,8 @@ └─LogicalFilter { predicate: (t.z > 0:Int32) AND (t.y > 0:Int32) AND (t.x > 0:Int32) AND (row_number <= 3:Int32) } └─LogicalProject { exprs: [t.x, t.y, t.z, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -164,7 +164,7 @@ └─LogicalAgg { aggs: [count(1:Int32), count(t.v1)] } └─LogicalProject { exprs: [1:Int32, t.v1] } └─LogicalFilter { predicate: (t.v2 > 1:Int32::Float64) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count(1:Int32), count(t.v1)] } └─LogicalProject { exprs: [1:Int32, t.v1] } @@ -177,7 +177,7 @@ LogicalProject { exprs: [t1.a, window_end] } └─LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.created_at, t1._row_id, t1._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: [t1.a, window_end] } └─LogicalScan { table: t1, columns: [t1.a, t1.created_at], predicate: IsNotNull(t1.created_at) } diff --git a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml index ccc0dda317c96..3bc5b2ff83ae3 100644 --- a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml @@ -6,7 +6,7 @@ LogicalProject { exprs: [t1.v1] } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [t1.v1, t1.v2] } - └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [v1, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [t1._row_id], pk_conflict: NoCheck } └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -17,10 +17,10 @@ logical_plan: |- LogicalProject { exprs: [t2.v3, t2.v4, t1.v1] } └─LogicalJoin { type: Inner, on: (t2.v3 = t1.v1), output: all } - ├─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [t1.v1] } - └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, v3], pk_columns: [t2._row_id, t1._row_id, v3], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(t2.v3, t2._row_id, t1._row_id) } @@ -39,7 +39,7 @@ └─LogicalProject { exprs: [t1.v1] } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [t1.v1, t1.v2] } - └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [v1, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [t1._row_id], pk_conflict: NoCheck } └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } @@ -53,7 +53,7 @@ └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } ├─LogicalAgg { group_key: [t1.x], aggs: [] } │ └─LogicalProject { exprs: [t1.x] } - │ └─LogicalScan { table: t1, columns: [t1.x, t1._row_id] } + │ └─LogicalScan { table: t1, columns: [t1.x, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [0.1:Decimal] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: Ensure we can bind CTE with aliases in both table name and columns @@ -65,17 +65,17 @@ └─LogicalJoin { type: Inner, on: (t1.x = t1.x), output: all } ├─LogicalShare { id: 2 } │ └─LogicalProject { exprs: [t1.x, t1.y] } - │ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + │ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [t1.x, t1.y] } - └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } - sql: | create table t (v int, c varchar); with dict(abbr, real) as (values ('cn', 'China'), ('us', 'United States')) select * from t join dict on t.c = dict.abbr; logical_plan: |- LogicalProject { exprs: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1] } └─LogicalJoin { type: Inner, on: (t.c = *VALUES*_0.column_0), output: all } - ├─LogicalScan { table: t, columns: [t.v, t.c, t._row_id] } + ├─LogicalScan { table: t, columns: [t.v, t.c, t._row_id, t._rw_timestamp] } └─LogicalShare { id: 1 } └─LogicalValues { rows: [['cn':Varchar, 'China':Varchar], ['us':Varchar, 'United States':Varchar]], schema: Schema { fields: [*VALUES*_0.column_0:Varchar, *VALUES*_0.column_1:Varchar] } } stream_plan: |- diff --git a/src/frontend/planner_test/tests/testdata/output/delete.yaml b/src/frontend/planner_test/tests/testdata/output/delete.yaml index 551cc6761ee82..67a61a6ea4e60 100644 --- a/src/frontend/planner_test/tests/testdata/output/delete.yaml +++ b/src/frontend/planner_test/tests/testdata/output/delete.yaml @@ -6,7 +6,8 @@ logical_plan: |- LogicalProject { exprs: [t.a, t.b, t.a, (t.a + t.b) as $expr1] } └─LogicalDelete { table: t, returning: true } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalProject { exprs: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.a, t.b, t.a, (t.a + t.b) as $expr1] } @@ -20,7 +21,8 @@ logical_plan: |- LogicalProject { exprs: [114514:Int32] } └─LogicalDelete { table: t, returning: true } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalProject { exprs: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [114514:Int32] } 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 0e800babc268a..6558e02efd2b8 100644 --- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml @@ -53,11 +53,11 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, _rw_timestamp ], 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 before: @@ -105,11 +105,11 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, _rw_timestamp ], 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 before: @@ -157,11 +157,11 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, _rw_timestamp ], 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 before: @@ -209,11 +209,11 @@ ├── StreamExchange Hash([0]) from 2 └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, k1, b__row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ v, bv, ak1.a._row_id, ak1.k1, bk1.b._row_id, _rw_timestamp ], 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 before: @@ -249,19 +249,24 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k1, a_v, a__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ a_k1, a_v, a__row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ a_k1, max(a_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ a_k1, max(a_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a.k1 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ max_v, a.k1, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk1_from_Ak1 before: @@ -290,13 +295,19 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ak1_k1, ak1_v, ak1_a__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ ak1_k1, ak1_v, ak1_a__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ ak1_k1, max(ak1_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ ak1_k1, max(ak1_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 + ├── columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ] + ├── 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, ak1.k1 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ max_v, ak1.k1, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk1_from_Ak1k2 before: @@ -325,13 +336,13 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ak1k2_k1, ak1k2_v, ak1k2_a__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ ak1k2_k1, ak1k2_v, ak1k2_a__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ ak1k2_k1, max(ak1k2_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ ak1k2_k1, max(ak1k2_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ vnode, k1, k2, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 { columns: [ vnode, k1, k2, a__row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ max_v, ak1k2.k1 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ max_v, ak1k2.k1, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk2_from_Ak1k2 before: @@ -366,13 +377,19 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ak1k2_k2, ak1k2_v, ak1k2_a__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ ak1k2_k2, ak1k2_v, ak1k2_a__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ ak1k2_k2, max(ak1k2_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ ak1k2_k2, max(ak1k2_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ vnode, k1, k2, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 + ├── columns: [ vnode, k1, k2, a__row_id, backfill_finished, row_count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4, 5 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 - Table 4294967294 { columns: [ max_v, ak1k2.k2 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ max_v, ak1k2.k2, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk1k2_from_Ak1k2 before: @@ -401,11 +418,11 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ak1k2_k1, ak1k2_k2, sum(ak1k2_v), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 0 { columns: [ ak1k2_k1, ak1k2_k2, sum(ak1k2_v), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 1 { columns: [ vnode, k1, k2, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, k1, k2, a__row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], 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 ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ sum_v, ak1k2.k1, ak1k2.k2, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - id: aggk1k2_from_Ak1 before: @@ -434,11 +451,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ak1_k1, ak1_k2, sum(ak1_v), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 0 { columns: [ ak1_k1, ak1_k2, sum(ak1_v), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 1 { columns: [ vnode, k1, a__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 + ├── columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ] + ├── 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, ak1.k1, ak1.k2 ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ sum_v, ak1.k1, ak1.k2, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - id: aggk1_from_aggk1 before: @@ -483,21 +506,21 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k1, count ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ a_k1, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ a_k1, max(count), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 3 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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_num, a.k1 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ max_num, a.k1, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk1_from_aggk1k2 before: @@ -519,7 +542,7 @@ └─LogicalProject { exprs: [count, a.k1] } └─LogicalAgg { group_key: [a.k1, a.k2], aggs: [count] } └─LogicalProject { exprs: [a.k1, a.k2] } - └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id] } + └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id, a._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [max(count)] } └─LogicalAgg { group_key: [a.k1], aggs: [max(count)] } @@ -560,21 +583,26 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k1, count, a_k2 ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ a_k1, count, a_k2, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ a_k1, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ a_k1, max(count), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ a_k1, a_k2, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 2 { columns: [ a_k1, a_k2, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 3 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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_num, a.k1 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ max_num, a.k1, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk2_from_aggk1k2 before: @@ -596,7 +624,7 @@ └─LogicalProject { exprs: [count, a.k2] } └─LogicalAgg { group_key: [a.k1, a.k2], aggs: [count] } └─LogicalProject { exprs: [a.k1, a.k2] } - └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id] } + └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id, a._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [max(count)] } └─LogicalAgg { group_key: [a.k2], aggs: [max(count)] } @@ -637,21 +665,26 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k2, count, a_k1 ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ a_k2, count, a_k1, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ a_k2, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ a_k2, max(count), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ a_k1, a_k2, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 2 { columns: [ a_k1, a_k2, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 3 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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_num, a.k2 ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ max_num, a.k2, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: aggk1k2_from_aggk1k2 before: @@ -695,21 +728,36 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k1, a_k2, count ], primary key: [ $0 ASC, $1 ASC, $2 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 + ├── columns: [ a_k1, a_k2, count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 DESC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 - Table 1 { columns: [ a_k1, a_k2, max(count), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 1 + ├── columns: [ a_k1, a_k2, max(count), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 - Table 2 { columns: [ a_k1, a_k2, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 2 { columns: [ a_k1, a_k2, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 3 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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_num, a.k1, a.k2 ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ max_num, a.k1, a.k2, _rw_timestamp ] + ├── primary key: [ $1 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 1, 2 ] + └── read pk prefix len hint: 2 - id: Ak1_join_aggk1_onk1 before: @@ -725,12 +773,12 @@ logical_plan: |- LogicalProject { exprs: [ak1.v, count] } └─LogicalJoin { type: Inner, on: (ak1.k1 = a.k1), output: all } - ├─LogicalScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.k3, ak1.v, ak1.a._row_id] } + ├─LogicalScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.k3, ak1.v, ak1.a._row_id, ak1._rw_timestamp] } └─LogicalShare { id: 4 } └─LogicalProject { exprs: [count, a.k1] } └─LogicalAgg { group_key: [a.k1], aggs: [count] } └─LogicalProject { exprs: [a.k1] } - └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id] } + └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id, a._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count] } @@ -773,33 +821,38 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ak1_k1, ak1_v, ak1_a__row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ ak1_k1, ak1_v, ak1_a__row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ ak1_k1, ak1_a__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ ak1_k1, ak1_a__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ a_k1, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ a_k1, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 - ├── columns: [ vnode, k1, a__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4 ] ├── 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 5 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a.k1 ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ v, bv, ak1.a._row_id, ak1.k1, a.k1, _rw_timestamp ] + ├── primary key: [ $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 2, 3 ] + └── read pk prefix len hint: 2 - id: aggk1_join_Ak1_onk1 before: @@ -819,8 +872,8 @@ │ └─LogicalProject { exprs: [count, a.k1] } │ └─LogicalAgg { group_key: [a.k1], aggs: [count] } │ └─LogicalProject { exprs: [a.k1] } - │ └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id] } - └─LogicalScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.k3, ak1.v, ak1.a._row_id] } + │ └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: ak1, columns: [ak1.k1, ak1.k2, ak1.k3, ak1.v, ak1.a._row_id, ak1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchLookupJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count], lookup table: ak1 } @@ -862,18 +915,18 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ a_k1, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ a_k1, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ ak1_k1, ak1_v, ak1_a__row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ ak1_k1, ak1_v, ak1_a__row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ ak1_k1, ak1_a__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ ak1_k1, ak1_a__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -881,14 +934,19 @@ └── vnode column idx: 0 Table 6 - ├── columns: [ vnode, k1, a__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, k1, a__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ v, bv, a.k1, ak1.a._row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ v, bv, a.k1, ak1.a._row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 2, 3 ] + └── read pk prefix len hint: 2 - id: aggk1_join_aggk1_onk1 before: @@ -913,12 +971,12 @@ │ └─LogicalProject { exprs: [count, a.k1] } │ └─LogicalAgg { group_key: [a.k1], aggs: [count] } │ └─LogicalProject { exprs: [a.k1] } - │ └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id] } + │ └─LogicalScan { table: a, columns: [a.k1, a.k2, a.k3, a.v, a._row_id, a._rw_timestamp] } └─LogicalShare { id: 8 } └─LogicalProject { exprs: [count, b.k1] } └─LogicalAgg { group_key: [b.k1], aggs: [count] } └─LogicalProject { exprs: [b.k1] } - └─LogicalScan { table: b, columns: [b.k1, b.k2, b.k3, b.v, b._row_id] } + └─LogicalScan { table: b, columns: [b.k1, b.k2, b.k3, b.v, b._row_id, b._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count] } @@ -964,35 +1022,40 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ a_k1, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ a_k1, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ b_k1, count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ b_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ b_k1, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ b_k1, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ a_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ b_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ b_k1, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ num, bv, a.k1, b.k1 ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ num, bv, a.k1, b.k1, _rw_timestamp ] + ├── primary key: [ $2 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 - sql: | create table t1 (row_id int, uid int, v int, created_at timestamp); @@ -1001,7 +1064,7 @@ LogicalProject { exprs: [t1.row_id, t1.uid, t1.v, t1.created_at, window_start, window_end] } └─LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalScan { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at, t1._row_id, t1._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalHopWindow { time_col: t1.created_at, slide: 00:15:00, size: 00:30:00, output: all } └─LogicalScan { table: t1, columns: [t1.row_id, t1.uid, t1.v, t1.created_at], predicate: IsNotNull(t1.created_at) } @@ -1026,10 +1089,10 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ row_id, uid, v, created_at, window_start, window_end, t1._row_id ] + ├── columns: [ row_id, uid, v, created_at, window_start, window_end, t1._row_id, _rw_timestamp ] ├── primary key: [ $6 ASC, $4 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 6 ] 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 3a45cddf1f6c3..22d4fbf9134ad 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 @@ -46,21 +46,21 @@ └── StreamSource { source: t, columns: [v1, v2, v3, _row_id] } { tables: [ Source: 3 ] } Table 0 - ├── columns: [ v1, min(v2), count(distinct v3), count ] + ├── columns: [ v1, min(v2), count(distinct v3), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ v1, v3, count_for_agg_call_1 ] + ├── columns: [ v1, v3, count_for_agg_call_1, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 2 Table 2 - ├── columns: [ vnode, offset ] + ├── columns: [ vnode, offset, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1 ] ├── distribution key: [ 0 ] @@ -68,14 +68,14 @@ └── vnode column idx: 0 Table 3 - ├── columns: [ partition_id, offset_info ] + ├── columns: [ partition_id, offset_info, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1 ] ├── distribution key: [] └── read pk prefix len hint: 1 Table 4294967294 - ├── columns: [ v1, min, agg ] + ├── columns: [ v1, min, agg, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0 ] @@ -116,17 +116,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ $expr1, max(t_b), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr1, max(t_b), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ window_start, max ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ window_start, max, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - sql: | create source t (a int, b int, tm timestamp, watermark for tm as tm - interval '5 minutes') with (connector = 'kinesis') FORMAT PLAIN ENCODE JSON; @@ -159,21 +159,21 @@ └── StreamSource { source: t, columns: [a, b, tm, _row_id] } { tables: [ Source: 3 ] } Table 0 - ├── columns: [ a, b, tm, _row_id ] + ├── columns: [ a, b, tm, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ a, b, tm, _row_id ] + ├── columns: [ a, b, tm, _row_id, _rw_timestamp ] ├── primary key: [ $2 ASC, $1 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 0 Table 2 - ├── columns: [ vnode, offset ] + ├── columns: [ vnode, offset, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1 ] ├── distribution key: [ 0 ] @@ -181,14 +181,14 @@ └── vnode column idx: 0 Table 3 - ├── columns: [ partition_id, offset_info ] + ├── columns: [ partition_id, offset_info, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1 ] ├── distribution key: [] └── read pk prefix len hint: 1 Table 4294967294 - ├── columns: [ lag, _row_id, b ] + ├── columns: [ lag, _row_id, b, _rw_timestamp ] ├── primary key: [ $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 2 ] diff --git a/src/frontend/planner_test/tests/testdata/output/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml index d19ec9882c42e..a7e0286a426f3 100644 --- a/src/frontend/planner_test/tests/testdata/output/except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/except.yaml @@ -54,38 +54,38 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t1_a, t1_b, t1_c, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ t1_a, t1_b, t1_c, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 1 - ├── columns: [ t1_a, t1_b, t1_c, t1__row_id ] + ├── columns: [ t1_a, t1_b, t1_c, t1__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 2 - ├── columns: [ t1_a, t1_b, t1_c, t1__row_id, _degree ] + ├── columns: [ t1_a, t1_b, t1_c, t1__row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 4 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 3 - ├── columns: [ t2_a, t2_b, t2_c, t2__row_id ] + ├── columns: [ t2_a, t2_b, t2_c, t2__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 4 - ├── columns: [ t2_a, t2_b, t2_c, t2__row_id, _degree ] + ├── columns: [ t2_a, t2_b, t2_c, t2__row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 4 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 5 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -93,14 +93,14 @@ └── vnode column idx: 0 Table 6 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ 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 } + Table 4294967294 { columns: [ a, b, c, _rw_timestamp ], 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: | create table t1 (a int, b numeric, c bigint, primary key(a)); @@ -155,42 +155,42 @@ └── BatchPlanNode Table 0 - ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ] + ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ t1_a, t1_b, t1_c ] + ├── columns: [ t1_a, t1_b, t1_c, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 2 - ├── columns: [ t1_a, t1_b, t1_c, _degree ] + ├── columns: [ t1_a, t1_b, t1_c, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 3 - ├── columns: [ t2_a, t2_b, t2_c ] + ├── columns: [ t2_a, t2_b, t2_c, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 4 - ├── columns: [ t2_a, t2_b, t2_c, _degree ] + ├── columns: [ t2_a, t2_b, t2_c, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 5 - ├── columns: [ vnode, a, backfill_finished, row_count ] + ├── columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -198,14 +198,19 @@ └── vnode column idx: 0 Table 6 - ├── columns: [ vnode, a, backfill_finished, row_count ] + ├── columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a, b, c ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ a, b, c, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - sql: | create table t1 (a int, b numeric, c bigint); diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index 41ab32c3910ae..13d6d8d0a7c5d 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -95,15 +95,15 @@ LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } + ├─LogicalScan { table: t1, columns: [v1, _row_id, _rw_timestamp] } + └─LogicalScan { table: t2, columns: [v2, _row_id, _rw_timestamp] } Predicate Push Down: LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } + ├─LogicalScan { table: t1, columns: [v1, _row_id, _rw_timestamp] } + └─LogicalScan { table: t2, columns: [v2, _row_id, _rw_timestamp] } Push down the calculation of inputs of join's condition: @@ -111,15 +111,15 @@ LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } + ├─LogicalScan { table: t1, columns: [v1, _row_id, _rw_timestamp] } + └─LogicalScan { table: t2, columns: [v2, _row_id, _rw_timestamp] } Predicate Push Down: LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } + ├─LogicalScan { table: t1, columns: [v1, _row_id, _rw_timestamp] } + └─LogicalScan { table: t2, columns: [v2, _row_id, _rw_timestamp] } Prune Columns: diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index 3ddc9d8fafc72..eacab421069a9 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -113,10 +113,10 @@ logical_plan: |- LogicalProject { exprs: [b.b2] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id] } + ├─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id, b._rw_timestamp] } └─LogicalProject { exprs: [2:Int32] } └─LogicalFilter { predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = CorrelatedInputRef { index: 0, correlated_id: 1 })) } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } batch_plan: |- BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } @@ -481,7 +481,7 @@ logical_plan: |- LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v1 >= Now) } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [v1, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [t._row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamDynamicFilter { predicate: (t.v1 >= now), output_watermarks: [t.v1], output: [t.v1, t._row_id], cleaned_by_watermark: true } @@ -495,7 +495,7 @@ logical_plan: |- LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v1 >= (Now - '00:00:02':Interval)) } - └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id, t._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [v1, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [t._row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamDynamicFilter { predicate: (t.v1 >= $expr1), output_watermarks: [t.v1], output: [t.v1, t._row_id], cleaned_by_watermark: true } diff --git a/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml b/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml index 896dad60c0fff..2b0c10cf89b3d 100644 --- a/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml +++ b/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml @@ -6,7 +6,7 @@ logical_plan: |- LogicalTopN { order: [t1.id ASC], limit: 2, offset: 0 } └─LogicalProject { exprs: [t1.id, t1.i] } - └─LogicalScan { table: t1, columns: [t1.id, t1.i] } + └─LogicalScan { table: t1, columns: [t1.id, t1.i, t1._rw_timestamp] } batch_plan: |- BatchTopN { order: [t1.id ASC], limit: 2, offset: 0 } └─BatchExchange { order: [], dist: Single } @@ -27,7 +27,7 @@ logical_plan: |- LogicalTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } └─LogicalProject { exprs: [t1.id, t1.i] } - └─LogicalScan { table: t1, columns: [t1.id, t1.i] } + └─LogicalScan { table: t1, columns: [t1.id, t1.i, t1._rw_timestamp] } batch_plan: |- BatchTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } └─BatchExchange { order: [], dist: Single } @@ -48,7 +48,7 @@ select cnt from v order by cnt; logical_plan: |- LogicalProject { exprs: [v.cnt] } - └─LogicalScan { table: v, columns: [v.cnt], cardinality: 0..=1 } + └─LogicalScan { table: v, columns: [v.cnt, v._rw_timestamp], cardinality: 0..=1 } batch_plan: |- BatchExchange { order: [v.cnt ASC], dist: Single } └─BatchSort { order: [v.cnt ASC] } diff --git a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml index 67a89e0878f9b..a6240c69f395f 100644 --- a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml @@ -215,12 +215,12 @@ BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchExchange { order: [], dist: Single } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id], lookup table: t1 } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id, t1._rw_timestamp], lookup table: t1 } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: |- BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } - └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id], lookup table: t1 } + └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id, t1._rw_timestamp], lookup table: t1 } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } - sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/insert.yaml b/src/frontend/planner_test/tests/testdata/output/insert.yaml index 26dcfca24eaee..7621987ac82c2 100644 --- a/src/frontend/planner_test/tests/testdata/output/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/output/insert.yaml @@ -239,7 +239,7 @@ └─LogicalInsert { table: t, returning: true, mapping: [0:0, 1:1] } └─LogicalAgg { group_key: [t.a, t.b], aggs: [] } └─LogicalProject { exprs: [t.a, t.b] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [a, b, a, (a + b) as $expr1] } @@ -292,7 +292,7 @@ logical_plan: |- LogicalInsert { table: t, mapping: [0:0], default: [1<-null:Int32] } └─LogicalProject { exprs: [t.b] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchInsert { table: t, mapping: [0:0], default: [1<-null:Int32] } @@ -317,7 +317,7 @@ LogicalInsert { table: t, mapping: [0:0], default: [1<-null:Int32] } └─LogicalProject { exprs: [t.a] } └─LogicalProject { exprs: [t.a] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchInsert { table: t, mapping: [0:0], default: [1<-null:Int32] } diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml index 678091bbb2e47..0f4bfbb623194 100644 --- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml +++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml @@ -54,38 +54,38 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t1_a, t1_b, t1_c, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ t1_a, t1_b, t1_c, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 1 - ├── columns: [ t1_a, t1_b, t1_c, t1__row_id ] + ├── columns: [ t1_a, t1_b, t1_c, t1__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 2 - ├── columns: [ t1_a, t1_b, t1_c, t1__row_id, _degree ] + ├── columns: [ t1_a, t1_b, t1_c, t1__row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 4 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 3 - ├── columns: [ t2_a, t2_b, t2_c, t2__row_id ] + ├── columns: [ t2_a, t2_b, t2_c, t2__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 4 - ├── columns: [ t2_a, t2_b, t2_c, t2__row_id, _degree ] + ├── columns: [ t2_a, t2_b, t2_c, t2__row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 4 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 5 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -93,14 +93,14 @@ └── vnode column idx: 0 Table 6 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ 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 } + Table 4294967294 { columns: [ a, b, c, _rw_timestamp ], 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: | create table t1 (a int, b numeric, c bigint, primary key(a)); @@ -155,42 +155,42 @@ └── BatchPlanNode Table 0 - ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ] + ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ t1_a, t1_b, t1_c ] + ├── columns: [ t1_a, t1_b, t1_c, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 2 - ├── columns: [ t1_a, t1_b, t1_c, _degree ] + ├── columns: [ t1_a, t1_b, t1_c, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 3 - ├── columns: [ t2_a, t2_b, t2_c ] + ├── columns: [ t2_a, t2_b, t2_c, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 4 - ├── columns: [ t2_a, t2_b, t2_c, _degree ] + ├── columns: [ t2_a, t2_b, t2_c, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 5 - ├── columns: [ vnode, a, backfill_finished, row_count ] + ├── columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -198,14 +198,19 @@ └── vnode column idx: 0 Table 6 - ├── columns: [ vnode, a, backfill_finished, row_count ] + ├── columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a, b, c ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ a, b, c, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - sql: | create table t1 (a int, b numeric, c bigint); diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index a1617d04e7d2e..aaa401e469d16 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -9,9 +9,9 @@ └─LogicalFilter { predicate: (t1.v1 = t2.v3) AND (t1.v1 = t3.v5) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } + │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } + └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id, t3._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id, t3._row_id) } @@ -30,8 +30,8 @@ logical_plan: |- LogicalProject { exprs: [t.v1, t.v1] } └─LogicalJoin { type: Inner, on: (t.v1 = t.v1), output: all } - ├─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + ├─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, t1v1], pk_columns: [t._row_id, t._row_id#1, t1v1], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) } @@ -240,10 +240,10 @@ logical_plan: |- LogicalProject { exprs: [t1.v1, t3.v2] } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t3, columns: [t3.v2, t3._row_id] } + ├─LogicalScan { table: t3, columns: [t3.v2, t3._row_id, t3._rw_timestamp] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v1), output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v1, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v1, t2._row_id, t2._rw_timestamp] } - name: Ensure correct binding of join with ON clause sql: | create table t1(v1 varchar); @@ -253,10 +253,10 @@ logical_plan: |- LogicalProject { exprs: [t3.v3, t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t3, columns: [t3.v3, t3._row_id] } + ├─LogicalScan { table: t3, columns: [t3.v3, t3._row_id, t3._rw_timestamp] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v2, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v2, t2._row_id, t2._rw_timestamp] } - name: Ensure correct binding with USING clause with left outer join sql: | create table t1(v1 varchar); @@ -266,10 +266,10 @@ logical_plan: |- LogicalProject { exprs: [t1.v1, t3.v2] } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t3, columns: [t3.v2, t3._row_id] } + ├─LogicalScan { table: t3, columns: [t3.v2, t3._row_id, t3._rw_timestamp] } └─LogicalJoin { type: LeftOuter, on: (t1.v1 = t2.v1), output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v1, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v1, t2._row_id, t2._rw_timestamp] } - name: Ensure correct binding with ON clause with left outer join sql: | create table t1(v1 varchar); @@ -279,10 +279,10 @@ logical_plan: |- LogicalProject { exprs: [t3.v3, t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t3, columns: [t3.v3, t3._row_id] } + ├─LogicalScan { table: t3, columns: [t3.v3, t3._row_id, t3._rw_timestamp] } └─LogicalJoin { type: LeftOuter, on: (t1.v1 = t2.v2), output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v2, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v2, t2._row_id, t2._rw_timestamp] } - name: Ensure that ON clause cannot reference correlated columns sql: | create table a(a1 int); @@ -301,10 +301,10 @@ logical_plan: |- LogicalProject { exprs: [a.a1, b.b1] } └─LogicalApply { type: Inner, on: true, correlated_id: 1 } - ├─LogicalScan { table: a, columns: [a.a1, a._row_id] } + ├─LogicalScan { table: a, columns: [a.a1, a._row_id, a._rw_timestamp] } └─LogicalProject { exprs: [b.b1] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } = b.b1) } - └─LogicalScan { table: b, columns: [b.b1, b._row_id] } + └─LogicalScan { table: b, columns: [b.b1, b._row_id, b._rw_timestamp] } - name: Ensure that natural joins bind the correct columns sql: | create table a(x int); @@ -315,9 +315,9 @@ LogicalProject { exprs: [a.x, c.y] } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: (a.x = b.x), output: all } - │ ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - │ └─LogicalScan { table: b, columns: [b.x, b._row_id] } - └─LogicalScan { table: c, columns: [c.y, c._row_id] } + │ ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + │ └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } + └─LogicalScan { table: c, columns: [c.y, c._row_id, c._rw_timestamp] } - name: Ensure that natural joins can disambiguate columns sql: | create table a(x int); @@ -326,8 +326,8 @@ logical_plan: |- LogicalProject { exprs: [a.x, a.x, b.x] } └─LogicalJoin { type: Inner, on: (a.x = b.x), output: all } - ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - └─LogicalScan { table: b, columns: [b.x, b._row_id] } + ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } - name: Ensure that natural joins bind the correct columns sql: | create table a(x int); @@ -338,9 +338,9 @@ LogicalProject { exprs: [a.x, c.y] } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: (a.x = b.x), output: all } - │ ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - │ └─LogicalScan { table: b, columns: [b.x, b._row_id] } - └─LogicalScan { table: c, columns: [c.y, c._row_id] } + │ ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + │ └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } + └─LogicalScan { table: c, columns: [c.y, c._row_id, c._rw_timestamp] } - name: Ensure that natural joins can disambiguate columns sql: | create table a(x int); @@ -349,8 +349,8 @@ logical_plan: |- LogicalProject { exprs: [a.x, a.x, b.x] } └─LogicalJoin { type: Inner, on: (a.x = b.x), output: all } - ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - └─LogicalScan { table: b, columns: [b.x, b._row_id] } + ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } - name: Ensure that natural joins bind the correct columns sql: | create table a(x int); @@ -361,9 +361,9 @@ LogicalProject { exprs: [a.x, c.y] } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: (a.x = b.x), output: all } - │ ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - │ └─LogicalScan { table: b, columns: [b.x, b._row_id] } - └─LogicalScan { table: c, columns: [c.y, c._row_id] } + │ ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + │ └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } + └─LogicalScan { table: c, columns: [c.y, c._row_id, c._rw_timestamp] } - name: Ensure that natural joins bind the correct columns sql: | create table a(x int); @@ -372,8 +372,8 @@ logical_plan: |- LogicalProject { exprs: [a.x] } └─LogicalJoin { type: Inner, on: (a.x = b.x), output: all } - ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - └─LogicalScan { table: b, columns: [b.x, b._row_id] } + ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } - name: Ensure that natural joins bind the correct columns sql: | create table a(x int); @@ -382,8 +382,8 @@ logical_plan: |- LogicalProject { exprs: [a.x] } └─LogicalJoin { type: LeftOuter, on: (a.x = b.x), output: all } - ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - └─LogicalScan { table: b, columns: [b.x, b._row_id] } + ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } - name: Ensure that natural joins bind the correct columns sql: | create table a(x int); @@ -392,8 +392,8 @@ logical_plan: |- LogicalProject { exprs: [b.x, a.x, b.x] } └─LogicalJoin { type: RightOuter, on: (a.x = b.x), output: all } - ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - └─LogicalScan { table: b, columns: [b.x, b._row_id] } + ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } - name: Ensure that natural joins bind the correct columns sql: | create table a(x int); @@ -402,8 +402,8 @@ logical_plan: |- LogicalProject { exprs: [Coalesce(a.x, b.x) as $expr1, a.x, b.x] } └─LogicalJoin { type: FullOuter, on: (a.x = b.x), output: all } - ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - └─LogicalScan { table: b, columns: [b.x, b._row_id] } + ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } - name: Ensure that natural joins work on materialized views sql: | create table t (a int, b int, c int); @@ -413,8 +413,8 @@ logical_plan: |- LogicalProject { exprs: [mv1.c, mv1.b, mv1.a, mv1.out1, mv2.out2] } └─LogicalJoin { type: Inner, on: (mv1.a = mv2.a) AND (mv1.b = mv2.b) AND (mv1.c = mv2.c), output: all } - ├─LogicalScan { table: mv1, columns: [mv1.a, mv1.b, mv1.c, mv1.t._row_id, mv1.out1] } - └─LogicalScan { table: mv2, columns: [mv2.a, mv2.b, mv2.c, mv2.t._row_id, mv2.out2] } + ├─LogicalScan { table: mv1, columns: [mv1.a, mv1.b, mv1.c, mv1.t._row_id, mv1.out1, mv1._rw_timestamp] } + └─LogicalScan { table: mv2, columns: [mv2.a, mv2.b, mv2.c, mv2.t._row_id, mv2.out2, mv2._rw_timestamp] } - name: Ensure that nested natural joins bind and disambiguate columns sql: | create table a(x int, y int); @@ -425,9 +425,9 @@ LogicalProject { exprs: [a.x, a.x, b.x, c.x] } └─LogicalJoin { type: Inner, on: (a.x = c.x), output: all } ├─LogicalJoin { type: Inner, on: (a.x = b.x), output: all } - │ ├─LogicalScan { table: a, columns: [a.x, a.y, a._row_id] } - │ └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id] } - └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id] } + │ ├─LogicalScan { table: a, columns: [a.x, a.y, a._row_id, a._rw_timestamp] } + │ └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id, b._rw_timestamp] } + └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id, c._rw_timestamp] } - name: Ensure that nested natural joins bind and disambiguate columns sql: | create table a(x int, y int); @@ -438,9 +438,9 @@ LogicalProject { exprs: [Coalesce(a.x, b.x, c.x) as $expr1, a.x, b.x, c.x] } └─LogicalJoin { type: FullOuter, on: (Coalesce(a.x, b.x) = c.x), output: all } ├─LogicalJoin { type: FullOuter, on: (a.x = b.x), output: all } - │ ├─LogicalScan { table: a, columns: [a.x, a.y, a._row_id] } - │ └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id] } - └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id] } + │ ├─LogicalScan { table: a, columns: [a.x, a.y, a._row_id, a._rw_timestamp] } + │ └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id, b._rw_timestamp] } + └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id, c._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [Coalesce(a.x, b.x, c.x) as $expr2, a.x, b.x, c.x] } └─LogicalJoin { type: FullOuter, on: ($expr1 = c.x), output: [a.x, b.x, c.x] } @@ -459,9 +459,9 @@ LogicalProject { exprs: [Coalesce(a.a, c.a) as $expr1, Coalesce(b.x, c.x) as $expr2, a.a, c.a, b.x, c.x] } └─LogicalJoin { type: FullOuter, on: (a.a = c.a) AND (b.x = c.x), output: all } ├─LogicalJoin { type: FullOuter, on: true, output: all } - │ ├─LogicalScan { table: a, columns: [a.a, a.y, a._row_id] } - │ └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id] } - └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id] } + │ ├─LogicalScan { table: a, columns: [a.a, a.y, a._row_id, a._rw_timestamp] } + │ └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id, b._rw_timestamp] } + └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id, c._rw_timestamp] } - name: Ensure error on non-existent USING col sql: | create table t1(v1 int, v2 int); @@ -485,11 +485,11 @@ LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8] } └─LogicalJoin { type: FullOuter, on: (t1.v2 = t3.v6) AND (t2.v4 = t4.v8), output: all } ├─LogicalJoin { type: Inner, on: (t1.v1 = t2.v3), output: all } - │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } └─LogicalJoin { type: Inner, on: (t3.v5 = t4.v7), output: all } - ├─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } - └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id] } + ├─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id, t3._rw_timestamp] } + └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id, t4._rw_timestamp] } - name: Ensure that we can correctly bind nested joins with ambiguous column names sql: | create table t1(x int); @@ -499,10 +499,10 @@ logical_plan: |- LogicalProject { exprs: [Coalesce(t1.x, t2.x, t3.x) as $expr1, Coalesce(t1.x, t2.x, t3.x) as $expr2, t1.x, t2.x, t3.x] } └─LogicalJoin { type: FullOuter, on: (t1.x = Coalesce(t2.x, t3.x)), output: all } - ├─LogicalScan { table: t1, columns: [t1.x, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1._row_id, t1._rw_timestamp] } └─LogicalJoin { type: FullOuter, on: (t2.x = t3.x), output: all } - ├─LogicalScan { table: t2, columns: [t2.x, t2._row_id] } - └─LogicalScan { table: t3, columns: [t3.x, t3._row_id] } + ├─LogicalScan { table: t2, columns: [t2.x, t2._row_id, t2._rw_timestamp] } + └─LogicalScan { table: t3, columns: [t3.x, t3._row_id, t3._rw_timestamp] } - name: Ensure that non-trivial ambiguous references can be resolved sql: | create table a(x int); @@ -513,8 +513,8 @@ └─LogicalProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr1, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr2, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr3] } └─LogicalFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) } └─LogicalJoin { type: FullOuter, on: (a.x = b.x), output: all } - ├─LogicalScan { table: a, columns: [a.x, a._row_id] } - └─LogicalScan { table: b, columns: [b.x, b._row_id] } + ├─LogicalScan { table: a, columns: [a.x, a._row_id, a._rw_timestamp] } + └─LogicalScan { table: b, columns: [b.x, b._row_id, b._rw_timestamp] } batch_plan: |- BatchProject { exprs: [$expr3, $expr2] } └─BatchExchange { order: [$expr2 ASC], dist: Single } @@ -550,8 +550,8 @@ └─LogicalProject { exprs: [test.a, test.b, test2.c, test.a] } └─LogicalFilter { predicate: (test.a = test2.a) AND (test.b <> test2.c) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: test, columns: [test.a, test.b, test._row_id] } - └─LogicalScan { table: test2, columns: [test2.a, test2.c, test2._row_id] } + ├─LogicalScan { table: test, columns: [test.a, test.b, test._row_id, test._rw_timestamp] } + └─LogicalScan { table: test2, columns: [test2.a, test2.c, test2._row_id, test2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [test.a, test.b, test2.c, test.a] } └─LogicalJoin { type: Inner, on: (test.a = test2.a) AND (test.b <> test2.c), output: [test.a, test.b, test2.c] } @@ -700,27 +700,32 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_src, t_dst, t_src_0, t_dst_0, t__row_id, t__row_id_0 ], primary key: [ $2 ASC, $0 ASC, $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 0 { columns: [ t_src, t_dst, t_src_0, t_dst_0, t__row_id, t__row_id_0, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC, $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 1 { columns: [ t_src, t_src_0, t__row_id, t__row_id_0, t_dst, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 1 ], read pk prefix len hint: 2 } + Table 1 { columns: [ t_src, t_src_0, t__row_id, t__row_id_0, t_dst, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - Table 2 { columns: [ t_dst, t__row_id ], primary key: [ $0 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ t_dst, t__row_id, _rw_timestamp ], primary key: [ $0 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ t_dst, t_dst_0, t__row_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 1 ], read pk prefix len hint: 2 } + Table 3 { columns: [ t_dst, t_dst_0, t__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - Table 4 { columns: [ t_src, t_dst, t__row_id ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4 { columns: [ t_src, t_dst, t__row_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 5 { columns: [ t_dst, t__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ t_dst, t__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ t_src, t_dst, t__row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ t_src, t_dst, t__row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - 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 7 { columns: [ t_src, t__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2 ], primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 3, 4, 5, 6 ], read pk prefix len hint: 6 } + Table 4294967294 + ├── columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2, _rw_timestamp ] + ├── primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 1, 3, 4, 5, 6 ] + └── read pk prefix len hint: 6 diff --git a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml index e1d18b0bd470c..ec07c7a0ed1ca 100644 --- a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml @@ -13,10 +13,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - │ │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - │ └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } - └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id] } + │ │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + │ │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } + │ └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id, t3._rw_timestamp] } + └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id, t4._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: all } @@ -61,10 +61,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - │ │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } - │ └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } - └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id] } + │ │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + │ │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } + │ └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id, t3._rw_timestamp] } + └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id, t4._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: all } @@ -108,11 +108,11 @@ LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8] } └─LogicalJoin { type: Inner, on: (t1.v2 = t3.v6), output: all } ├─LogicalJoin { type: Inner, on: (t1.v1 = t2.v3), output: all } - │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + │ ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + │ └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } └─LogicalJoin { type: Inner, on: (t3.v5 = t4.v7), output: all } - ├─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } - └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id] } + ├─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id, t3._rw_timestamp] } + └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id, t4._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: all } @@ -155,12 +155,12 @@ logical_plan: |- LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v3), output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } └─LogicalJoin { type: Inner, on: (t2.v4 = t3.v6), output: all } - ├─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } └─LogicalJoin { type: Inner, on: (t3.v5 = t4.v7), output: all } - ├─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } - └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id] } + ├─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id, t3._rw_timestamp] } + └─LogicalScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id, t4._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: all } diff --git a/src/frontend/planner_test/tests/testdata/output/limit.yaml b/src/frontend/planner_test/tests/testdata/output/limit.yaml index 22fb2add9d30c..a7b5c0d7a7f48 100644 --- a/src/frontend/planner_test/tests/testdata/output/limit.yaml +++ b/src/frontend/planner_test/tests/testdata/output/limit.yaml @@ -5,7 +5,7 @@ logical_plan: |- LogicalLimit { limit: 4, offset: 0 } └─LogicalProject { exprs: [(t.v + t.v) as $expr1] } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(t.v + t.v) as $expr1] } └─LogicalLimit { limit: 4, offset: 0 } @@ -16,7 +16,7 @@ logical_plan: |- LogicalLimit { limit: 9223372036854775807, offset: 4 } └─LogicalProject { exprs: [t.v] } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } - sql: | create table t (v int); select * from ( select v+v from t limit 5 ) limit 4; @@ -25,7 +25,7 @@ └─LogicalProject { exprs: [$expr1] } └─LogicalLimit { limit: 5, offset: 0 } └─LogicalProject { exprs: [(t.v + t.v) as $expr1] } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalLimit { limit: 4, offset: 0 } └─LogicalProject { exprs: [(t.v + t.v) as $expr1] } @@ -37,28 +37,28 @@ logical_plan: |- LogicalLimit { limit: 4, offset: 0 } └─LogicalProject { exprs: [t.v] } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } - sql: | create table t (v int); select * from t offset 3 fetch first 4 rows only; logical_plan: |- LogicalLimit { limit: 4, offset: 3 } └─LogicalProject { exprs: [t.v] } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } - sql: | create table t (v int); select * from t fetch next rows only; logical_plan: |- LogicalLimit { limit: 1, offset: 0 } └─LogicalProject { exprs: [t.v] } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } - sql: | create table t (v int); select * from t order by v fetch next 2 rows with ties; logical_plan: |- LogicalTopN { order: [t.v ASC], limit: 2, offset: 0, with_ties: true } └─LogicalProject { exprs: [t.v] } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } - sql: | -- Should be equivalent to above create table t (v int); @@ -144,7 +144,7 @@ └─LogicalProject { exprs: [count] } └─LogicalAgg { aggs: [count] } └─LogicalProject { exprs: [] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } batch_plan: |- BatchLimit { limit: 1, offset: 0 } └─BatchSimpleAgg { aggs: [sum0(count)] } diff --git a/src/frontend/planner_test/tests/testdata/output/logical_scan_predicate_eliminate.yaml b/src/frontend/planner_test/tests/testdata/output/logical_scan_predicate_eliminate.yaml index 684e4c9144d01..c81a7baec2702 100644 --- a/src/frontend/planner_test/tests/testdata/output/logical_scan_predicate_eliminate.yaml +++ b/src/frontend/planner_test/tests/testdata/output/logical_scan_predicate_eliminate.yaml @@ -10,7 +10,7 @@ logical_plan: |- LogicalProject { exprs: [t1.c1] } └─LogicalFilter { predicate: Not((t1.c1 > 1:Int32)) AND (t1.c1 > 1:Int32) } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: 'BatchValues { rows: [] }' - id: non_trivial_optimizable_case before: @@ -29,7 +29,7 @@ logical_plan: |- LogicalProject { exprs: [t1.c1, t1.c2, t1.c3] } └─LogicalFilter { predicate: (t1.c1 > 1:Int32) AND (t1.c2 > 2:Int32) AND (t1.c3 > 3:Int32) AND Not((t1.c1 < 1:Int32)) AND Not((t1.c2 < 2:Int32)) AND Not((t1.c3 < 3:Int32)) AND Not((t1.c1 > 1:Int32)) AND Not((t1.c2 > 2:Int32)) AND Not((t1.c3 > 3:Int32)) } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: 'BatchValues { rows: [] }' - id: corner_case before: @@ -46,7 +46,7 @@ logical_plan: |- LogicalProject { exprs: [t1.c1, t1.c2, t1.c3] } └─LogicalFilter { predicate: Not((t1.c1 > 1:Int32)) AND Not((t1.c2 > 2:Int32)) AND Not((t1.c3 > 3:Int32)) AND ((t1.c1 > 1:Int32) OR (t1.c2 > 2:Int32)) AND ((t1.c2 > 2:Int32) OR (t1.c3 > 3:Int32)) AND ((t1.c3 > 3:Int32) OR (t1.c1 > 1:Int32)) AND (((t1.c1 > 1:Int32) OR (t1.c2 > 2:Int32)) OR (t1.c3 > 3:Int32)) } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: Not((t1.c1 > 1:Int32)) AND Not((t1.c2 > 2:Int32)) AND Not((t1.c3 > 3:Int32)) AND ((t1.c1 > 1:Int32) OR (t1.c2 > 2:Int32)) AND ((t1.c2 > 2:Int32) OR (t1.c3 > 3:Int32)) AND ((t1.c3 > 3:Int32) OR (t1.c1 > 1:Int32)) AND (((t1.c1 > 1:Int32) OR (t1.c2 > 2:Int32)) OR (t1.c3 > 3:Int32)) } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index d6b90da0a8c1a..d33c3d1b6bcae 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -62,9 +62,9 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, date_time, bid._row_id ], primary key: [ $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, date_time, bid._row_id, _rw_timestamp ], primary key: [ $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - id: nexmark_q1 before: @@ -97,9 +97,9 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, date_time, bid._row_id ], primary key: [ $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, date_time, bid._row_id, _rw_timestamp ], primary key: [ $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - id: nexmark_q2 before: @@ -126,9 +126,15 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] + ├── 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: [ auction, price, bid._row_id ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, price, bid._row_id, _rw_timestamp ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - id: nexmark_q3 before: @@ -185,19 +191,24 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ auction_id, auction_seller ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction_id, auction_seller, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction_seller, auction_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ auction_seller, auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ person_id, person_name, person_city, person_state ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ person_id, person_name, person_city, person_state, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ person_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ person_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], 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: [ name, city, state, id, auction.seller, person.id ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3, 4 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ name, city, state, id, auction.seller, person.id, _rw_timestamp ] + ├── primary key: [ $3 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5 ] + ├── distribution key: [ 3, 4 ] + └── read pk prefix len hint: 2 - id: nexmark_q4 before: @@ -270,25 +281,25 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ auction_category, sum(max(bid_price)), count(max(bid_price)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction_category, sum(max(bid_price)), count(max(bid_price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction_id, auction_category, bid_price, bid__row_id ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC ], value indices: [ 0, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ auction_id, auction_category, bid_price, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC ], value indices: [ 0, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 2 { columns: [ auction_id, auction_category, max(bid_price), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ auction_id, auction_category, max(bid_price), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ auction_id, auction_date_time, auction_expires, auction_category ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, auction_date_time, auction_expires, auction_category, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ bid_auction, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 8 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ category, avg ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ category, avg, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q5 before: @@ -330,7 +341,7 @@ │ └─LogicalProject { exprs: [window_start, bid.auction] } │ └─LogicalHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: all } │ └─LogicalFilter { predicate: IsNotNull(bid.date_time) } - │ └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + │ └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } └─LogicalProject { exprs: [max(count), window_start] } └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } └─LogicalProject { exprs: [window_start, count] } @@ -339,7 +350,7 @@ └─LogicalProject { exprs: [bid.auction, window_start] } └─LogicalHopWindow { time_col: bid.date_time, slide: 00:00:02, size: 00:00:10, output: all } └─LogicalFilter { predicate: IsNotNull(bid.date_time) } - └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [bid.auction, count] } @@ -410,23 +421,29 @@ StreamNoOp └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ bid_auction, window_start, count ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ bid_auction, window_start, count, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ window_start, bid_auction, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ window_start, bid_auction, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ window_start, max(count) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ window_start, max(count), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ window_start, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ window_start, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - 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 4 { columns: [ bid_auction, window_start, count, _rw_timestamp ], 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, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] + ├── 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: [ 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 6 { columns: [ window_start, count, bid_auction, _rw_timestamp ], 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 7 { columns: [ window_start, max(count), count, _rw_timestamp ], 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 ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, num, window_start, window_start#1, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 2 } - id: nexmark_q6_group_top1 before: @@ -488,23 +505,23 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ auction_seller, bid_price, bid_date_time, auction_id, sum, count ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction_seller, bid_price, bid_date_time, auction_id, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction_id, auction_seller, bid_price, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 1 { columns: [ auction_id, auction_seller, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 2 { columns: [ auction_id, auction_date_time, auction_expires, auction_seller ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction_id, auction_date_time, auction_expires, auction_seller, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { 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: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 7 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ seller, avg, auction.id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ seller, avg, auction.id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: nexmark_q6 before: @@ -580,23 +597,23 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ auction_seller, bid_price, bid_date_time, auction_id, sum, count ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction_seller, bid_price, bid_date_time, auction_id, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction_id, auction_seller, bid_price, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 1 { columns: [ auction_id, auction_seller, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 2 { columns: [ auction_id, auction_date_time, auction_expires, auction_seller ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction_id, auction_date_time, auction_expires, auction_seller, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { 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: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 7 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ seller, avg, auction.id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ seller, avg, auction.id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: nexmark_q7 before: @@ -673,21 +690,26 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 0 { columns: [ bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 1 { columns: [ bid_price, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ bid_price, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr1, max(bid_price), $expr2 ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr1, max(bid_price), $expr2, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - 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 3 { columns: [ max(bid_price), $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ $expr1, max(bid_price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr1, max(bid_price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, price, bidder, date_time, bid._row_id, $expr1 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, price, bidder, date_time, bid._row_id, $expr1, _rw_timestamp ] + ├── primary key: [ $4 ASC, $5 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5 ] + ├── distribution key: [ 1 ] + └── read pk prefix len hint: 3 - id: nexmark_q8 before: @@ -786,24 +808,34 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name) ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 + ├── columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name), _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 0, 1, 2 ] + └── read pk prefix len hint: 3 - Table 1 { columns: [ person_id, $expr1, $expr2, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 1 { columns: [ person_id, $expr1, $expr2, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 2 { columns: [ auction_seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ auction_seller, $expr3, $expr4, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ auction_seller, $expr3, $expr4, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ auction_seller, $expr3, $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4 + ├── columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 3, 4 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 3 - Table 5 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ auction_seller, $expr3, $expr4, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 6 { columns: [ auction_seller, $expr3, $expr4, count, _rw_timestamp ], 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, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } Table 4294967294 - ├── columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ] + ├── columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 0, 2, 3 ] @@ -828,11 +860,11 @@ └─LogicalFilter { predicate: (row_number <= 1:Int32) } └─LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.date_time, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY auction.id ORDER BY bid.price DESC, bid.date_time ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + └─LogicalProject { exprs: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, auction._rw_timestamp, bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } └─LogicalFilter { predicate: (auction.id = bid.auction) AND (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category] } - └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + ├─LogicalScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, auction._rw_timestamp] } + └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [bid.price DESC, bid.date_time ASC], limit: 1, offset: 0, group_key: [auction.id] } └─LogicalJoin { type: Inner, on: (auction.id = bid.auction) AND (bid.date_time >= auction.date_time) AND (bid.date_time <= auction.expires), output: all } @@ -877,25 +909,35 @@ └── BatchPlanNode Table 0 - ├── columns: [ auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category, bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id ] + ├── columns: [ auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category, bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $11 DESC, $12 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 + ├── columns: [ auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 2 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - 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 4 { columns: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], 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, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, bid._row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - id: nexmark_q10 before: @@ -923,9 +965,9 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, date_time, date, time, 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 } + Table 4294967294 { columns: [ auction, bidder, price, date_time, date, time, bid._row_id, _rw_timestamp ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } - id: nexmark_q11 before: @@ -981,11 +1023,11 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ bid_bidder, $expr1, $expr2, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ bid_bidder, $expr1, $expr2, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ bidder, bid_count, window_start, window_end ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ bidder, bid_count, window_start, window_end, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } - id: nexmark_q13 before: @@ -1036,12 +1078,12 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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, key, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, key, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, date_time, value, bid._row_id, $expr1, side_input.key ] + ├── columns: [ auction, bidder, price, date_time, value, bid._row_id, $expr1, side_input.key, _rw_timestamp ] ├── primary key: [ $5 ASC, $6 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 5, 6 ] @@ -1096,9 +1138,9 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, bid._row_id, _rw_timestamp ], 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 before: @@ -1149,19 +1191,19 @@ └── BatchPlanNode Table 0 - ├── columns: [ $expr1, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32)) ] + ├── columns: [ $expr1, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1, bid_bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ $expr1, bid_bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - 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 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, _rw_timestamp ], 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, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + 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, _rw_timestamp ], 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_q15_split_distinct_agg before: @@ -1213,17 +1255,17 @@ └── BatchPlanNode Table 0 - ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder_expanded) filter((flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction_expanded) filter((flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder_expanded) filter((flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction_expanded) filter((flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1_expanded, bid_bidder_expanded, bid_auction_expanded, flag, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 1 { columns: [ $expr1_expanded, bid_bidder_expanded, bid_auction_expanded, flag, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 2 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1278,19 +1320,19 @@ └── BatchPlanNode Table 0 - ├── columns: [ bid_channel, $expr1, max($expr2), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32)) ] + ├── columns: [ bid_channel, $expr1, max($expr2), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count(distinct bid_bidder), count(distinct bid_bidder) filter((bid_price < 10000:Int32)), count(distinct bid_bidder) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_bidder) filter((bid_price >= 1000000:Int32)), count(distinct bid_auction), count(distinct bid_auction) filter((bid_price < 10000:Int32)), count(distinct bid_auction) filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count(distinct bid_auction) filter((bid_price >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ bid_channel, $expr1, bid_bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8 ], 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 1 { columns: [ bid_channel, $expr1, bid_bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8, _rw_timestamp ], 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 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 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, _rw_timestamp ], 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, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + 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, _rw_timestamp ], 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_q16_split_distinct_agg before: @@ -1343,20 +1385,20 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ bid_channel_expanded, $expr1_expanded, max($expr2_expanded), bid_bidder_expanded, bid_auction_expanded, flag ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ bid_channel_expanded, $expr1_expanded, max($expr2_expanded), bid_bidder_expanded, bid_auction_expanded, flag, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 1 - ├── columns: [ bid_channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder_expanded) filter((flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction_expanded) filter((flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ bid_channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder_expanded) filter((flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction_expanded) filter((flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 2 { columns: [ bid_channel_expanded, $expr1_expanded, bid_bidder_expanded, bid_auction_expanded, flag, max($expr2_expanded), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + Table 2 { columns: [ bid_channel_expanded, $expr1_expanded, bid_bidder_expanded, bid_auction_expanded, flag, max($expr2_expanded), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } - Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1406,15 +1448,15 @@ └── BatchPlanNode Table 0 - ├── columns: [ bid_auction, $expr1, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price) ] + ├── columns: [ bid_auction, $expr1, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price, _rw_timestamp ], 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 before: @@ -1429,8 +1471,8 @@ └─LogicalFilter { predicate: (row_number <= 1:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY bid.bidder, bid.auction ORDER BY bid.date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } - └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } + └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [bid.date_time DESC], limit: 1, offset: 0, group_key: [bid.bidder, bid.auction] } @@ -1455,15 +1497,15 @@ └── BatchPlanNode Table 0 - ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid__row_id ] + ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid__row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, channel, url, date_time, extra, bid._row_id ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, bid._row_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: nexmark_q18_rank before: @@ -1478,8 +1520,8 @@ └─LogicalFilter { predicate: (rank <= 1:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, rank] } └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY bid.bidder, bid.auction ORDER BY bid.date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } - └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } + └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [bid.date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bid.bidder, bid.auction] } @@ -1504,15 +1546,15 @@ └── BatchPlanNode Table 0 - ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid__row_id ] + ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid__row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, channel, url, date_time, extra, bid._row_id ], primary key: [ $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, bid._row_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 3 } - id: nexmark_q19_no_rank before: @@ -1526,8 +1568,8 @@ └─LogicalFilter { predicate: (row_number <= 10:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } - └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } + └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchGroupTopN { order: [bid.price DESC], limit: 10, offset: 0, group_key: [bid.auction] } @@ -1552,15 +1594,15 @@ └── BatchPlanNode Table 0 - ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid_p_time, bid__row_id ] + ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid_p_time, bid__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 DESC, $8 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, channel, url, date_time, extra, p_time, bid._row_id ], primary key: [ $0 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, p_time, bid._row_id, _rw_timestamp ], primary key: [ $0 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: nexmark_q19 before: @@ -1574,8 +1616,8 @@ └─LogicalFilter { predicate: (row_number <= 10:Int32) } └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } - └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id] } + └─LogicalProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } + └─LogicalScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid.extra, bid.p_time, bid._row_id, bid._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY bid.auction ORDER BY bid.price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1604,22 +1646,27 @@ └── BatchPlanNode Table 0 - ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid_p_time, bid__row_id, row_number ] + ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid_p_time, bid__row_id, row_number, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 DESC, $8 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid_p_time, bid__row_id ] + ├── columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid_extra, bid_p_time, bid__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 DESC, $8 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, channel, url, date_time, extra, p_time, bid._row_id, rank_number ], primary key: [ $0 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, p_time, bid._row_id, rank_number, _rw_timestamp ] + ├── primary key: [ $0 ASC, $8 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 - id: nexmark_q20 before: @@ -1669,20 +1716,20 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ bid_auction, bid_bidder, bid_price, bid_channel, bid_url, bid_date_time, bid__row_id, _rw_timestamp ], 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 1 { 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 1 { columns: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction_id, auction_item_name, auction_description, auction_initial_bid, auction_reserve, auction_date_time, auction_expires, auction_seller, auction_category, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id, auction.id ] + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id, auction.id, _rw_timestamp ] ├── primary key: [ $14 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 14 ] @@ -1723,9 +1770,9 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ auction, bidder, price, channel, channel_id, bid._row_id ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, bid._row_id, _rw_timestamp ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - id: nexmark_q22 before: @@ -1757,9 +1804,9 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id, _rw_timestamp ], 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 before: @@ -1825,22 +1872,34 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction_id, auction_item_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ bid_auction, max(bid_price) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ bid_auction, max(bid_price), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ bid_auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ bid_auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 + ├── columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ] + ├── 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: [ bid_auction, max(bid_price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ bid_auction, max(bid_price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] + ├── 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: [ auction_id, auction_item_name, current_highest_bid, bid.auction ] + ├── columns: [ auction_id, auction_item_name, current_highest_bid, bid.auction, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -1942,31 +2001,31 @@ └── BatchPlanNode Table 0 - ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction) ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), _rw_timestamp ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, auction_item_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ bid_auction, bid__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ bid_auction, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ bid_auction, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 - ├── columns: [ vnode, id, backfill_finished, row_count ] + ├── columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -1974,26 +2033,31 @@ └── vnode column idx: 0 Table 8 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ sum0(sum0(count)), sum0(count(bid_auction)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ sum0(sum0(count)), sum0(count(bid_auction)), count, _rw_timestamp ], 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 10 { columns: [ bid_auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 11 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ auction_id, auction_item_name, bid_count ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - id: nexmark_q103 before: @@ -2059,33 +2123,33 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction_id, auction_item_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ bid_auction ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ bid_auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ bid_auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ bid_auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 - ├── columns: [ vnode, id, backfill_finished, row_count ] + ├── columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ bid_auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q104 before: @@ -2151,33 +2215,33 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction_id, auction_item_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ bid_auction ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ bid_auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ bid_auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ bid_auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 - ├── columns: [ vnode, id, backfill_finished, row_count ] + ├── columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ bid_auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q105 before: @@ -2249,14 +2313,14 @@ └── BatchPlanNode Table 0 - ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [] └── read pk prefix len hint: 0 Table 1 - ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1, _rw_timestamp ] ├── primary key: [ $3 ASC, $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -2264,22 +2328,22 @@ └── vnode column idx: 3 Table 2 - ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, auction_item_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ bid_auction, bid__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ bid_auction, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ bid_auction, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 - ├── columns: [ vnode, id, backfill_finished, row_count ] + ├── columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -2287,14 +2351,19 @@ └── vnode column idx: 0 Table 8 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ auction_id, auction_item_name, bid_count ], primary key: [ $2 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 2 - id: nexmark_q106 before: @@ -2378,29 +2447,35 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ min(max(bid_price)), $expr1 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ min(max(bid_price)), $expr1, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ min(min(max(bid_price))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ min(min(max(bid_price))), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ $expr1, max(bid_price), auction_id ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 + ├── columns: [ $expr1, max(bid_price), auction_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 2 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 - Table 3 { columns: [ $expr1, min(max(bid_price)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ $expr1, min(max(bid_price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4 { columns: [ auction_id, bid_price, bid__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, bid_price, bid__row_id, _rw_timestamp ], 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: [ auction_id, max(bid_price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction_id, max(bid_price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ auction_id, auction_date_time, auction_expires ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ auction_id, auction_date_time, auction_expires, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ auction_id, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ bid_auction, bid_price, bid_date_time, bid__row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { 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 9 { columns: [ bid_auction, bid__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, id, backfill_finished, row_count, _rw_timestamp ], 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, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], 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: [ min_final ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ min_final, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 35713c9682a35..472bb56b2c12e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -65,10 +65,15 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] } - Table 0 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ partition_id, offset_info, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 Table 4294967294 - ├── columns: [ auction, bidder, price, date_time, _row_id ] + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] ├── primary key: [ $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] @@ -101,10 +106,15 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] } - Table 0 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ partition_id, offset_info, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 Table 4294967294 - ├── columns: [ auction, bidder, price, date_time, _row_id ] + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] ├── primary key: [ $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] @@ -134,9 +144,14 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] } - Table 0 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 0 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, price, _row_id ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ auction, price, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 - id: nexmark_q3 before: @@ -194,29 +209,29 @@ └── StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } { tables: [ Source: 5 ] } Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $7 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 7 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ seller, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ seller, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 - ├── columns: [ id, name, email_address, credit_card, city, state, date_time, extra, _row_id ] + ├── columns: [ id, name, email_address, credit_card, city, state, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $8 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1 ] + ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1, _rw_timestamp ] ├── primary key: [ $4 ASC, $6 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 4, 5, 6 ] @@ -293,52 +308,67 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 7 ] } Table 0 - ├── columns: [ category, sum(max(price)), count(max(price)), count ] + ├── columns: [ category, sum(max(price)), count(max(price)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ id, category, max(price), count ] + ├── columns: [ id, category, max(price), count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 2 Table 2 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 3 - ├── columns: [ id, _row_id, _degree ] + ├── columns: [ id, _row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 4 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 5 - ├── columns: [ auction, _row_id, _degree ] + ├── columns: [ auction, _row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 + ├── columns: [ partition_id, offset_info, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 + ├── columns: [ partition_id, offset_info, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 - Table 4294967294 { columns: [ category, avg ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ category, avg, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - id: nexmark_q5 before: @@ -465,24 +495,29 @@ StreamNoOp └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ auction, window_start, count ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ window_start, auction, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ window_start, auction, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ window_start, max(count) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ window_start, max(count), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ window_start, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ window_start, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ auction, window_start, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 4 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ window_start, count, auction ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 + ├── columns: [ window_start, count, auction, _rw_timestamp ] + ├── 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 7 { columns: [ window_start, max(count), count, _rw_timestamp ], 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 ] + ├── columns: [ auction, num, window_start, window_start#1, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 2 ] @@ -564,28 +599,28 @@ StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 7 ] } - Table 0 { columns: [ seller, price, date_time, id, sum, count ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ seller, price, date_time, id, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ id, seller, price, date_time, _row_id, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 1 { columns: [ id, seller, price, date_time, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 2 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ seller, avg, id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ seller, avg, id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: nexmark_q7 before: @@ -668,20 +703,25 @@ StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ auction, bidder, price, date_time, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ price, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ price, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr1, max(price), $expr2 ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr1, max(price), $expr2, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 3 { columns: [ max(price), $expr1, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ max(price), $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr1, max(price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr1, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, price, bidder, date_time, _row_id, $expr1 ] + ├── columns: [ auction, price, bidder, date_time, _row_id, $expr1, _rw_timestamp ] ├── primary key: [ $4 ASC, $5 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 1 ] @@ -782,24 +822,24 @@ └── StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { tables: [ Source: 7 ] } - Table 0 { columns: [ id, name, $expr1, $expr2 ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + Table 0 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } - Table 1 { columns: [ id, $expr1, $expr2, name, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 1 { columns: [ id, $expr1, $expr2, name, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 2 { columns: [ seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ seller, $expr3, $expr4, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ seller, $expr3, $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ id, name, $expr1, $expr2 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 4 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 6 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4 ] + ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 0, 1, 2, 3 ] @@ -874,31 +914,31 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 6 ] } Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time_0, _row_id, _row_id_0 ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time_0, _row_id, _row_id_0, _rw_timestamp ] ├── primary key: [ $0 ASC, $11 DESC, $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: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1 ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [ 0 ] @@ -926,10 +966,10 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] } - Table 0 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 0 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, bidder, price, date_time, date, time, _row_id ] + ├── columns: [ auction, bidder, price, date_time, date, time, _row_id, _rw_timestamp ] ├── primary key: [ $6 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 6 ] @@ -1027,9 +1067,9 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] } - Table 0 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 0 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id, _rw_timestamp ], 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 before: @@ -1080,19 +1120,19 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 3 ] } Table 0 - ├── columns: [ $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)) ] + ├── columns: [ $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1, bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ $expr1, bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr1, 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 2 { columns: [ $expr1, auction, count_for_agg_call_8, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 3 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + 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, _rw_timestamp ], 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_q15_split_distinct_agg before: @@ -1144,17 +1184,17 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 2 ] } Table 0 - ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 1 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1212,25 +1252,25 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 3 ] } Table 0 - ├── columns: [ $expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count ] + ├── columns: [ $expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ $expr1_expanded, _vnode, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, _vnode, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [] ├── read pk prefix len hint: 2 └── vnode column idx: 1 - Table 2 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 2 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 3 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1286,19 +1326,19 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 3 ] } Table 0 - ├── columns: [ channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)) ] + ├── columns: [ channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ channel, $expr1, bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8 ], 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 1 { columns: [ channel, $expr1, bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8, _rw_timestamp ], 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 2 { columns: [ channel, $expr1, 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 2 { columns: [ channel, $expr1, auction, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, count_for_agg_call_12, _rw_timestamp ], 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: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 3 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + 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, _rw_timestamp ], 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_q16 before: @@ -1351,20 +1391,20 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 3 ] } - Table 0 { columns: [ channel_expanded, $expr1_expanded, max($expr2_expanded), bidder_expanded, auction_expanded, flag ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ channel_expanded, $expr1_expanded, max($expr2_expanded), bidder_expanded, auction_expanded, flag, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 1 - ├── columns: [ channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 2 { columns: [ channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag, max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + Table 2 { columns: [ channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag, max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } - Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 3 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1414,15 +1454,20 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 1 ] } Table 0 - ├── columns: [ auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price) ] + ├── columns: [ auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + Table 4294967294 + ├── columns: [ auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price, _rw_timestamp ] + ├── 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 before: @@ -1465,16 +1510,16 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 1 ] } Table 0 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] @@ -1521,16 +1566,16 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 1 ] } Table 0 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] @@ -1580,23 +1625,23 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 2 ] } Table 0 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, row_number ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, row_number, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number, _rw_timestamp ] ├── primary key: [ $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0 ] @@ -1653,20 +1698,20 @@ └── StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { tables: [ Source: 5 ] } - Table 0 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ], primary key: [ $0 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1, _rw_timestamp ] ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 14, 15 ] @@ -1708,9 +1753,9 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] } - Table 0 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 0 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id, _rw_timestamp ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - id: nexmark_q22 before: @@ -1737,9 +1782,14 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] } - Table 0 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 0 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _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 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _row_id, _rw_timestamp ] + ├── 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 before: @@ -1804,26 +1854,26 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 6 ] } Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ auction, max(price) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction, max(price), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ auction, max(price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction ] + ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction, _rw_timestamp ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0, 3 ] @@ -1928,35 +1978,45 @@ StreamNoOp └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ id, item_name, count(auction) ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ id, item_name, count(auction), _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ id, item_name, count(auction), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 + ├── columns: [ id, item_name, count(auction), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 Table 3 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ auction, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 8 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 9 { columns: [ sum0(sum0(count)), sum0(count(auction)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ sum0(sum0(count)), sum0(count(auction)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 10 { columns: [ auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ auction, count, _rw_timestamp ], 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 ] + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0 ] @@ -2026,26 +2086,26 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 6 ] } Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ auction ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction_id, auction_item_name, _row_id ] + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 2 ] @@ -2115,26 +2175,26 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 6 ] } Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ auction ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction_id, auction_item_name, _row_id ] + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 2 ] @@ -2206,41 +2266,51 @@ StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 8 ] } - Table 0 { columns: [ id, item_name, count(auction), $expr1 ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 1 - ├── columns: [ id, item_name, count(auction), $expr1 ] + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] ├── primary key: [ $3 ASC, $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] ├── read pk prefix len hint: 1 └── vnode column idx: 3 - Table 2 { columns: [ id, item_name, count(auction), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ id, item_name, count(auction), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } Table 3 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 8 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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: 3 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 3 - id: nexmark_q106 before: @@ -2328,16 +2398,21 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 10 ] } Table 0 - ├── columns: [ min(max(price)), $expr1 ] + ├── columns: [ min(max(price)), $expr1, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 0, 1 ] ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 { columns: [ min(min(max(price))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ min(min(max(price))), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 2 - ├── columns: [ $expr1, max(price), id ] + ├── columns: [ $expr1, max(price), id, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 1, 2 ] ├── distribution key: [ 2 ] @@ -2345,46 +2420,61 @@ └── vnode column idx: 0 Table 3 - ├── columns: [ $expr1, min(max(price)), count ] + ├── columns: [ $expr1, min(max(price)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2 ] ├── distribution key: [] ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 4 { columns: [ id, max(price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 + ├── columns: [ id, max(price), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 5 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 6 - ├── columns: [ id, _row_id, _degree ] + ├── columns: [ id, _row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 7 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 8 - ├── columns: [ auction, _row_id, _degree ] + ├── columns: [ auction, _row_id, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 9 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 9 + ├── columns: [ partition_id, offset_info, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 - Table 10 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 10 + ├── columns: [ partition_id, offset_info, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 - Table 4294967294 { columns: [ min_final ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ min_final, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml index db3ec1e70e650..469d94ca87ff3 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml @@ -70,10 +70,15 @@ ├── tables: [ SourceBackfill: 0 ] └── Upstream - Table 0 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 4294967294 - ├── columns: [ auction, bidder, price, date_time, _row_id ] + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] ├── primary key: [ $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] @@ -108,10 +113,15 @@ ├── tables: [ SourceBackfill: 0 ] └── Upstream - Table 0 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 4294967294 - ├── columns: [ auction, bidder, price, date_time, _row_id ] + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] ├── primary key: [ $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] @@ -143,9 +153,14 @@ ├── tables: [ SourceBackfill: 0 ] └── Upstream - Table 0 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, price, _row_id ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ auction, price, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 - id: nexmark_q3 before: @@ -206,29 +221,29 @@ └── Upstream Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $7 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 7 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ seller, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ seller, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 - ├── columns: [ id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $11 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 5 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1 ] + ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1, _rw_timestamp ] ├── primary key: [ $4 ASC, $6 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 4, 5, 6 ] @@ -306,33 +321,33 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } └── Upstream - Table 0 { columns: [ category, sum(max(price)), count(max(price)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ category, sum(max(price)), count(max(price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ id, category, max(price), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ id, category, max(price), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } Table 2 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 5 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 7 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ category, avg ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ category, avg, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q5 before: @@ -461,24 +476,29 @@ StreamNoOp └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ auction, window_start, count ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ window_start, auction, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ window_start, auction, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ window_start, max(count) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ window_start, max(count), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ window_start, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ window_start, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ auction, window_start, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 4 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 5 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 6 { columns: [ window_start, count, auction ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 + ├── columns: [ window_start, count, auction, _rw_timestamp ] + ├── 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 7 { columns: [ window_start, max(count), count, _rw_timestamp ], 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 ] + ├── columns: [ auction, num, window_start, window_start#1, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 2 ] @@ -562,33 +582,33 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } └── Upstream - Table 0 { columns: [ seller, price, date_time, id, sum, count ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ seller, price, date_time, id, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ id, seller, price, date_time, _row_id, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 1 { columns: [ id, seller, price, date_time, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 2 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 5 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 7 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ seller, avg, id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ seller, avg, id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: nexmark_q7 before: @@ -673,20 +693,25 @@ StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ auction, bidder, price, date_time, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ price, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ price, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr1, max(price), $expr2 ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr1, max(price), $expr2, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 3 { columns: [ max(price), $expr1, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ max(price), $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 5 { columns: [ $expr1, max(price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr1, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, price, bidder, date_time, _row_id, $expr1 ] + ├── columns: [ auction, price, bidder, date_time, _row_id, $expr1, _rw_timestamp ] ├── primary key: [ $4 ASC, $5 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 1 ] @@ -789,23 +814,28 @@ └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } └── Upstream - Table 0 { columns: [ id, name, $expr1, $expr2 ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + Table 0 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } - Table 1 { columns: [ id, $expr1, $expr2, name, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 1 { columns: [ id, $expr1, $expr2, name, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 2 { columns: [ seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ seller, $expr3, $expr4, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ seller, $expr3, $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ id, name, $expr1, $expr2 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 4 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 5 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 6 { columns: [ seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 6 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 7 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 4294967294 + ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 1, 2, 3 ] + └── read pk prefix len hint: 4 - id: nexmark_q9 before: @@ -878,36 +908,36 @@ └── Upstream Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time_0, _row_id, _row_id_0 ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time_0, _row_id, _row_id_0, _rw_timestamp ] ├── primary key: [ $0 ASC, $11 DESC, $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: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 3 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 6 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1 ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [ 0 ] @@ -937,10 +967,15 @@ ├── tables: [ SourceBackfill: 0 ] └── Upstream - Table 0 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 4294967294 - ├── columns: [ auction, bidder, price, date_time, date, time, _row_id ] + ├── columns: [ auction, bidder, price, date_time, date, time, _row_id, _rw_timestamp ] ├── primary key: [ $6 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 6 ] @@ -1039,9 +1074,9 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } └── Upstream - Table 0 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id, _rw_timestamp ], 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 before: @@ -1093,19 +1128,19 @@ └── Upstream Table 0 - ├── columns: [ $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)) ] + ├── columns: [ $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1, bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ $expr1, bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr1, 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 2 { columns: [ $expr1, auction, count_for_agg_call_8, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 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 } + 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, _rw_timestamp ], 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_q15_split_distinct_agg before: @@ -1158,17 +1193,17 @@ └── Upstream Table 0 - ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 1 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 2 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 2 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1227,25 +1262,25 @@ └── Upstream Table 0 - ├── columns: [ $expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count ] + ├── columns: [ $expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ $expr1_expanded, _vnode, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, _vnode, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [] ├── read pk prefix len hint: 2 └── vnode column idx: 1 - Table 2 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 2 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 3 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1302,19 +1337,19 @@ └── Upstream Table 0 - ├── columns: [ channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)) ] + ├── columns: [ channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ channel, $expr1, bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8 ], 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 1 { columns: [ channel, $expr1, bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8, _rw_timestamp ], 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 2 { columns: [ channel, $expr1, 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 2 { columns: [ channel, $expr1, auction, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, count_for_agg_call_12, _rw_timestamp ], 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: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 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 } + 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, _rw_timestamp ], 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_q16 before: @@ -1368,20 +1403,20 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } └── Upstream - Table 0 { columns: [ channel_expanded, $expr1_expanded, max($expr2_expanded), bidder_expanded, auction_expanded, flag ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ channel_expanded, $expr1_expanded, max($expr2_expanded), bidder_expanded, auction_expanded, flag, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 1 - ├── columns: [ channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 2 { columns: [ channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag, max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + Table 2 { columns: [ channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag, max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } - Table 3 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 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 } + 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, _rw_timestamp ], 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 } with_config_map: RW_FORCE_SPLIT_DISTINCT_AGG: 'true' @@ -1432,15 +1467,20 @@ └── Upstream Table 0 - ├── columns: [ auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price) ] + ├── columns: [ auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 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 } + Table 4294967294 + ├── columns: [ auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price, _rw_timestamp ] + ├── 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 before: @@ -1485,16 +1525,16 @@ └── Upstream Table 0 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] @@ -1542,16 +1582,16 @@ └── Upstream Table 0 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 1, 0 ] @@ -1602,23 +1642,23 @@ └── Upstream Table 0 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, row_number ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, row_number, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 2 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 2 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number, _rw_timestamp ] ├── primary key: [ $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0 ] @@ -1677,25 +1717,25 @@ └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } └── Upstream - Table 0 { columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ], primary key: [ $0 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 5 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1, _rw_timestamp ] ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 14, 15 ] @@ -1738,9 +1778,9 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } └── Upstream - Table 0 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id, _rw_timestamp ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - id: nexmark_q22 before: @@ -1768,9 +1808,14 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } └── Upstream - Table 0 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _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 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _row_id, _rw_timestamp ] + ├── 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 before: @@ -1838,26 +1883,26 @@ └── Upstream Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ auction, max(price) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction, max(price), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 5 { columns: [ auction, max(price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction ] + ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction, _rw_timestamp ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0, 3 ] @@ -1965,34 +2010,39 @@ StreamNoOp └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ id, item_name, count(auction) ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ id, item_name, count(auction), _rw_timestamp ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ id, item_name, count(auction), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ id, item_name, count(auction), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } Table 3 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ auction, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 8 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 9 { columns: [ sum0(sum0(count)), sum0(count(auction)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ sum0(sum0(count)), sum0(count(auction)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 10 { columns: [ auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ auction, count, _rw_timestamp ], 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 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 - id: nexmark_q103 before: @@ -2060,25 +2110,30 @@ └── Upstream Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ auction ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 5 { columns: [ auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0, 2 ] + └── read pk prefix len hint: 2 - id: nexmark_q104 before: @@ -2146,25 +2201,30 @@ └── Upstream Table 0 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ auction ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ auction, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 5 { columns: [ auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0, 2 ] + └── read pk prefix len hint: 2 - id: nexmark_q105 before: @@ -2235,41 +2295,51 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 8 ] } └── Upstream - Table 0 { columns: [ id, item_name, count(auction), $expr1 ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 1 - ├── columns: [ id, item_name, count(auction), $expr1 ] + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] ├── primary key: [ $3 ASC, $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] ├── read pk prefix len hint: 1 └── vnode column idx: 3 - Table 2 { columns: [ id, item_name, count(auction), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ id, item_name, count(auction), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } Table 3 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 8 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 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: 3 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 3 - id: nexmark_q106 before: @@ -2358,37 +2428,43 @@ └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 10 ] } └── Upstream - Table 0 { columns: [ min(max(price)), $expr1 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ min(max(price)), $expr1, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ min(min(max(price))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ min(min(max(price))), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ $expr1, max(price), id ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 + ├── columns: [ $expr1, max(price), id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 2 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 - Table 3 { columns: [ $expr1, min(max(price)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ $expr1, min(max(price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4 { columns: [ id, max(price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ id, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 - ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ id, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 - ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id ] + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 8 { columns: [ auction, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 10 { columns: [ partition_id, backfill_progress ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 10 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ min_final ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ min_final, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml index d5d948e5b507c..b93b35fbb7b3e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml @@ -94,19 +94,24 @@ └── StreamNow { output: [now] } { tables: [ Now: 3 ] } Table 0 - ├── columns: [ event_type, person, auction, bid, $expr1, _row_id ] + ├── columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ] ├── primary key: [ $4 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 5 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 3 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, date_time, _row_id ], primary key: [ $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 4 ] + └── read pk prefix len hint: 1 - id: nexmark_q2 before: @@ -138,15 +143,15 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 3 ] } - Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 3 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, price, _row_id ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, price, _row_id, _rw_timestamp ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - id: nexmark_q4 before: @@ -239,48 +244,58 @@ └── StreamNow { output: [now] } { tables: [ Now: 10 ] } Table 0 - ├── columns: [ $expr5, sum(max($expr8)), count(max($expr8)), count ] + ├── columns: [ $expr5, sum(max($expr8)), count(max($expr8)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ $expr2, $expr5, $expr8, _row_id, _row_id_0 ] + ├── columns: [ $expr2, $expr5, $expr8, _row_id, _row_id_0, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC ] ├── value indices: [ 0, 2, 3, 4 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 2 - Table 2 { columns: [ $expr2, $expr5, max($expr8), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 + ├── columns: [ $expr2, $expr5, max($expr8), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 Table 3 - ├── columns: [ $expr2, $expr3, $expr4, $expr5, _row_id ] + ├── columns: [ $expr2, $expr3, $expr4, $expr5, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr7, $expr8, $expr9, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 + ├── columns: [ $expr7, $expr8, $expr9, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 6 { columns: [ $expr7, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr7, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 8 - ├── columns: [ event_type, auction, bid, $expr1, _row_id ] + ├── columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ] ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] └── read pk prefix len hint: 1 - Table 9 { columns: [ $expr6 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ $expr6, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 10 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 10 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ category, avg ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ category, avg, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q5 before: @@ -389,35 +404,40 @@ StreamNoOp └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ $expr3, window_start, count ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr3, window_start, count, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ window_start, $expr3, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ window_start, $expr3, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ window_start, max(count) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ window_start, max(count), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ window_start, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ window_start, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr3, window_start, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 4 { columns: [ $expr3, window_start, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 5 - ├── columns: [ event_type, person, auction, bid, $expr1, _row_id ] + ├── columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ] ├── primary key: [ $4 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 5 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 6 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 8 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 9 { columns: [ window_start, count, $expr3 ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 + ├── columns: [ window_start, count, $expr3, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 10 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ window_start, max(count), count, _rw_timestamp ], 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 ] + ├── columns: [ auction, num, window_start, window_start#1, _rw_timestamp ] ├── primary key: [ $0 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 2 ] @@ -530,27 +550,27 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr6], output_watermarks: [$expr6] } └── StreamNow { output: [now] } { tables: [ Now: 9 ] } - Table 0 { columns: [ $expr5, $expr8, $expr9, $expr2, sum, count ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr5, $expr8, $expr9, $expr2, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, $expr5, $expr8, $expr9, _row_id, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 1 { columns: [ $expr2, $expr5, $expr8, $expr9, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr2, $expr3, $expr4, $expr5, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, $expr3, $expr4, $expr5, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr7, $expr8, $expr9, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr7, $expr8, $expr9, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr7, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr7, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 7 { columns: [ event_type, auction, bid, $expr1, _row_id ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 7 { columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 8 { columns: [ $expr6 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ $expr6, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 9 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ seller, avg, $expr2 ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ seller, avg, $expr2, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: nexmark_q7 before: @@ -643,28 +663,38 @@ StreamProject { exprs: [(TumbleStart($expr6, '00:00:10':Interval) + '00:00:10':Interval) as $expr7, $expr5, _row_id] } └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ $expr3, $expr4, $expr5, $expr6, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr5, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr5, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr7, max($expr5), $expr8 ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr7, max($expr5), $expr8, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 3 { columns: [ max($expr5), $expr7, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ max($expr5), $expr7, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 4 + ├── columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ] + ├── primary key: [ $4 ASC, $5 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5 ] + ├── distribution key: [ 5 ] + └── read pk prefix len hint: 1 - Table 5 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 7 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 8 { columns: [ $expr7, $expr5, _row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ $expr7, $expr5, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ $expr7, max($expr5), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ $expr7, max($expr5), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, price, bidder, date_time, _row_id, $expr7 ] + ├── columns: [ auction, price, bidder, date_time, _row_id, $expr7, _rw_timestamp ] ├── primary key: [ $4 ASC, $5 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 1 ] @@ -772,22 +802,22 @@ └── StreamFilter { predicate: (event_type = 1:Int32) } └── StreamExchange NoShuffle from 4 - Table 0 { columns: [ $expr2, $expr3, $expr5, $expr6 ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + Table 0 { columns: [ $expr2, $expr3, $expr5, $expr6, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } - Table 1 { columns: [ $expr2, $expr5, $expr6, $expr3, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 1 { columns: [ $expr2, $expr5, $expr6, $expr3, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 2 { columns: [ $expr8, $expr9, $expr10 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ $expr8, $expr9, $expr10, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ $expr8, $expr9, $expr10, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 3 { columns: [ $expr8, $expr9, $expr10, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ $expr2, $expr3, $expr5, $expr6 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 4 { columns: [ $expr2, $expr3, $expr5, $expr6, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr8, $expr9, $expr10 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 6 { columns: [ $expr8, $expr9, $expr10, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 4294967294 - ├── columns: [ id, name, starttime, $expr6, $expr8, $expr9, $expr10 ] + ├── columns: [ id, name, starttime, $expr6, $expr8, $expr9, $expr10, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 0, 1, 2, 3 ] @@ -869,25 +899,30 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr11], output_watermarks: [$expr11] } └── StreamNow { output: [now] } { tables: [ Now: 8 ] } - Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr10, $expr12, $expr13, $expr14, $expr15, _row_id, _row_id_0 ], primary key: [ $0 ASC, $11 DESC, $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: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr10, $expr12, $expr13, $expr14, $expr15, _row_id, _row_id_0, _rw_timestamp ] + ├── primary key: [ $0 ASC, $11 DESC, $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: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr10, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr10, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr12, $expr13, $expr14, $expr15, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr12, $expr13, $expr14, $expr15, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr12, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr12, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ event_type, auction, bid, $expr1, _row_id ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 6 { columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 7 { columns: [ $expr11 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ $expr11, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 8 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1 ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: nexmark_q11 before: @@ -989,15 +1024,15 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 3 ] } - Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 3 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } - id: nexmark_q18 before: @@ -1040,17 +1075,22 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 4 ] } - Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, _row_id ], primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 0 + ├── columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 - Table 1 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 1 { columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 2 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 3 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: nexmark_q18_rank before: @@ -1093,17 +1133,27 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 4 ] } - Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, _row_id ], primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 0 + ├── columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 - Table 1 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 1 { columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 2 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 3 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ], primary key: [ $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 3 - id: nexmark_q19 before: @@ -1147,19 +1197,19 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 5 ] } - Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr1, _row_id, row_number ], primary key: [ $0 ASC, $2 DESC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr1, _row_id, row_number, _rw_timestamp ], primary key: [ $0 ASC, $2 DESC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr1, _row_id ], primary key: [ $0 ASC, $2 DESC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr9, $expr1, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 DESC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 2 { columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, p_time, _row_id, rank_number ], primary key: [ $0 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, p_time, _row_id, rank_number, _rw_timestamp ], primary key: [ $0 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: nexmark_q20 before: @@ -1231,24 +1281,24 @@ └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, _row_id, _rw_timestamp ], 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 1 { columns: [ $expr3, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr3, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr9, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr9, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr9, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr9, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ event_type, auction, bid, $expr1, _row_id ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 4 { columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 7 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1, _rw_timestamp ] ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 14, 15 ] @@ -1298,15 +1348,15 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 3 ] } - Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 3 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id, _rw_timestamp ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - id: nexmark_q22 before: @@ -1346,15 +1396,20 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 3 ] } - Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 0 { columns: [ event_type, person, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 3 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _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 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _row_id, _rw_timestamp ] + ├── 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 before: @@ -1442,28 +1497,28 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { tables: [ Now: 9 ] } - Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr5, max($expr6) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr5, max($expr6), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr5, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr5, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr5, $expr6, _row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr5, $expr6, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr5, max($expr6), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr5, max($expr6), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ event_type, auction, bid, $expr1, _row_id ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 7 { columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 8 { columns: [ $expr4 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ $expr4, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 9 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr5 ] + ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr5, _rw_timestamp ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0, 3 ] @@ -1591,39 +1646,49 @@ StreamNoOp └── StreamExchange NoShuffle from 4 - Table 0 { columns: [ $expr2, $expr3, count($expr5) ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ $expr2, $expr3, count($expr5), _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr6 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr6, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ $expr2, $expr3, count($expr5), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 + ├── columns: [ $expr2, $expr3, count($expr5), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 - Table 3 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr5, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr5, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr5, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr5, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 8 - ├── columns: [ event_type, auction, bid, $expr1, _row_id ] + ├── columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ] ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] └── read pk prefix len hint: 1 - Table 9 { columns: [ $expr4 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ $expr4, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 10 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 10 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 11 { columns: [ sum0(sum0(count)), sum0(count($expr5)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 11 { columns: [ sum0(sum0(count)), sum0(count($expr5)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 12 { columns: [ $expr5, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ $expr5, count, _rw_timestamp ], 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 ] + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0 ] @@ -1713,31 +1778,41 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { tables: [ Now: 8 ] } - Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ $expr2, $expr3, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 + ├── columns: [ $expr2, _row_id, _degree, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 2 { columns: [ $expr5 ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr5, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr5, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr5, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr5, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr5, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 - ├── columns: [ event_type, auction, bid, $expr1, _row_id ] + ├── columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ] ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] └── read pk prefix len hint: 1 - Table 7 { columns: [ $expr4 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ $expr4, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 8 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction_id, auction_item_name, _row_id ] + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 2 ] @@ -1827,31 +1902,41 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { tables: [ Now: 8 ] } - Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ $expr2, $expr3, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 + ├── columns: [ $expr2, _row_id, _degree, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 2 { columns: [ $expr5 ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr5, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr5, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr5, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr5, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr5, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 - ├── columns: [ event_type, auction, bid, $expr1, _row_id ] + ├── columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ] ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] └── read pk prefix len hint: 1 - Table 7 { columns: [ $expr4 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 7 { columns: [ $expr4, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 8 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 8 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 4294967294 - ├── columns: [ auction_id, auction_item_name, _row_id ] + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 2 ] @@ -1945,35 +2030,50 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { tables: [ Now: 10 ] } - Table 0 { columns: [ $expr2, $expr3, count($expr5), $expr6 ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ $expr2, $expr3, count($expr5), $expr6, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 1 - ├── columns: [ $expr2, $expr3, count($expr5), $expr6 ] + ├── columns: [ $expr2, $expr3, count($expr5), $expr6, _rw_timestamp ] ├── primary key: [ $3 ASC, $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] ├── read pk prefix len hint: 1 └── vnode column idx: 3 - Table 2 { columns: [ $expr2, $expr3, count($expr5), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ $expr2, $expr3, count($expr5), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr5, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr5, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr5, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr5, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 8 { columns: [ event_type, auction, bid, $expr1, _row_id ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 8 + ├── columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ] + ├── primary key: [ $3 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 4 ] + └── read pk prefix len hint: 1 - Table 9 { columns: [ $expr4 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ $expr4, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 10 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 10 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 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: 3 } + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 3 - id: nexmark_q106 before: @@ -2081,12 +2181,17 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr5], output_watermarks: [$expr5] } └── StreamNow { output: [now] } { tables: [ Now: 13 ] } - Table 0 { columns: [ min(max($expr7)), $expr9 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ min(max($expr7)), $expr9, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 1 { columns: [ min(min(max($expr7))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ min(min(max($expr7))), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ $expr9, max($expr7), $expr2 ] + ├── columns: [ $expr9, max($expr7), $expr2, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 1, 2 ] ├── distribution key: [ 2 ] @@ -2094,7 +2199,7 @@ └── vnode column idx: 0 Table 3 - ├── columns: [ $expr9, min(max($expr7)), count ] + ├── columns: [ $expr9, min(max($expr7)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2 ] ├── distribution key: [] @@ -2102,44 +2207,59 @@ └── vnode column idx: 0 Table 4 - ├── columns: [ $expr2, $expr7, _row_id, _row_id_0 ] + ├── columns: [ $expr2, $expr7, _row_id, _row_id_0, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 DESC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 5 { columns: [ $expr2, max($expr7), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 + ├── columns: [ $expr2, max($expr7), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 6 - ├── columns: [ $expr2, $expr3, $expr4, _row_id ] + ├── columns: [ $expr2, $expr3, $expr4, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 7 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 + ├── columns: [ $expr2, _row_id, _degree, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 8 - ├── columns: [ $expr6, $expr7, $expr8, _row_id ] + ├── columns: [ $expr6, $expr7, $expr8, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 9 { columns: [ $expr6, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 + ├── columns: [ $expr6, _row_id, _degree, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 10 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 10 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 11 - ├── columns: [ event_type, auction, bid, $expr1, _row_id ] + ├── columns: [ event_type, auction, bid, $expr1, _row_id, _rw_timestamp ] ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 4 ] └── read pk prefix len hint: 1 - Table 12 { columns: [ $expr5 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 12 { columns: [ $expr5, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 13 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 13 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ min_final ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ min_final, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index f065ba33c252d..dc34896b1be00 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -55,11 +55,11 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 1 ] } - Table 0 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, date_time, _row_id ], primary key: [ $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ], primary key: [ $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4 ], read pk prefix len hint: 1 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -98,11 +98,11 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 1 ] } - Table 0 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, price, _row_id ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, price, _row_id, _rw_timestamp ], primary key: [ $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -184,19 +184,19 @@ └── StreamFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr3, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr3, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr4, $expr5, $expr6, $expr7, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr4, $expr5, $expr6, $expr7, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr4, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr4, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1 ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 4, 5, 6 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1, _rw_timestamp ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 4, 5, 6 ], read pk prefix len hint: 3 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -295,23 +295,23 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr4, sum(max($expr6)), count(max($expr6)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr4, sum(max($expr6)), count(max($expr6)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, $expr4, max($expr6), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ $expr2, $expr4, max($expr6), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr2, $expr1, $expr3, $expr4, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, $expr1, $expr3, $expr4, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr5, $expr6, $expr1, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr5, $expr6, $expr1, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr5, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr5, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ category, avg ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ category, avg, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -454,25 +454,25 @@ StreamNoOp └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ $expr2, window_start, count ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, window_start, count, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ window_start, $expr2, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ window_start, $expr2, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ window_start, max(count) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ window_start, max(count), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ window_start, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ window_start, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr2, window_start, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 4 { columns: [ $expr2, window_start, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 5 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 7 { columns: [ window_start, count, $expr2 ], 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, count, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ window_start, max(count), count, _rw_timestamp ], 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 ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, num, window_start, window_start#1, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 2 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck, watermark_columns: [window_start(hidden)] } @@ -601,23 +601,23 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 4 - Table 0 { columns: [ $expr4, $expr6, $expr1, $expr2, sum, count ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr4, $expr6, $expr1, $expr2, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, $expr4, $expr6, $expr1, _row_id, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 1 { columns: [ $expr2, $expr4, $expr6, $expr1, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr2, $expr1, $expr3, $expr4, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, $expr1, $expr3, $expr4, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr5, $expr6, $expr1, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr5, $expr6, $expr1, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr5, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr5, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 7 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ seller, avg, $expr2 ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ seller, avg, $expr2, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } eowc_stream_plan: |- StreamMaterialize { columns: [seller, avg, $expr2(hidden)], stream_key: [$expr2, seller], pk_columns: [$expr2, seller], pk_conflict: NoCheck } @@ -745,21 +745,21 @@ StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr2, $expr3, $expr4, $expr1, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, $expr4, $expr1, _row_id, _rw_timestamp ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr4, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr4, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr5, max($expr4), $expr6 ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr5, max($expr4), $expr6, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 3 { columns: [ max($expr4), $expr5, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ max($expr4), $expr5, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr5, max($expr4), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr5, max($expr4), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 4, 5 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5, _rw_timestamp ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 4, 5 ], read pk prefix len hint: 3 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -904,23 +904,23 @@ └── StreamFilter { predicate: (event_type = 1:Int32) } └── StreamExchange NoShuffle from 4 - Table 0 { columns: [ $expr3, $expr4, $expr2, $expr5 ], primary key: [ $2 ASC, $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + Table 0 { columns: [ $expr3, $expr4, $expr2, $expr5, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } - Table 1 { columns: [ $expr2, $expr5, $expr3, $expr4, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 0, 1 ], read pk prefix len hint: 3 } + Table 1 { columns: [ $expr2, $expr5, $expr3, $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 0, 1 ], read pk prefix len hint: 3 } - Table 2 { columns: [ $expr7, $expr6, $expr8 ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ $expr7, $expr6, $expr8, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ $expr6, $expr8, $expr7, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 0, 1 ], read pk prefix len hint: 3 } + Table 3 { columns: [ $expr6, $expr8, $expr7, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 0, 1 ], read pk prefix len hint: 3 } - Table 4 { columns: [ $expr3, $expr4, $expr2, $expr5 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 4 { columns: [ $expr3, $expr4, $expr2, $expr5, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } - Table 5 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 7 { columns: [ $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 7 { columns: [ $expr7, $expr6, $expr8, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } eowc_stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime] } @@ -1055,21 +1055,21 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, $expr8, $expr9, $expr10, $expr11, $expr12, $expr1_0, _row_id, _row_id_0 ], primary key: [ $0 ASC, $11 DESC, $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: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, $expr8, $expr9, $expr10, $expr11, $expr12, $expr1_0, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $11 DESC, $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: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, $expr8, $expr9, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, $expr8, $expr9, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr10, $expr11, $expr12, $expr1, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr10, $expr11, $expr12, $expr1, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr10, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr10, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 6 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1 ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0 ], read pk prefix len hint: 1 } eowc_stream_plan: |- StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -1125,11 +1125,11 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 1 ] } - Table 0 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, date_time, date, time, _row_id ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, date_time, date, time, _row_id, _rw_timestamp ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -1236,11 +1236,11 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 1 ] } - Table 0 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -1308,21 +1308,21 @@ └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 4 ] } Table 0 - ├── columns: [ $expr2, count, count filter(($expr3 < 10000:Int32)), count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count filter(($expr3 >= 1000000:Int32)), count(distinct $expr4), count(distinct $expr4) filter(($expr3 < 10000:Int32)), count(distinct $expr4) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr4) filter(($expr3 >= 1000000:Int32)), count(distinct $expr5), count(distinct $expr5) filter(($expr3 < 10000:Int32)), count(distinct $expr5) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr5) filter(($expr3 >= 1000000:Int32)) ] + ├── columns: [ $expr2, count, count filter(($expr3 < 10000:Int32)), count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count filter(($expr3 >= 1000000:Int32)), count(distinct $expr4), count(distinct $expr4) filter(($expr3 < 10000:Int32)), count(distinct $expr4) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr4) filter(($expr3 >= 1000000:Int32)), count(distinct $expr5), count(distinct $expr5) filter(($expr3 < 10000:Int32)), count(distinct $expr5) filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count(distinct $expr5) filter(($expr3 >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr2, $expr4, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ $expr2, $expr4, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr2, $expr5, 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 2 { columns: [ $expr2, $expr5, count_for_agg_call_8, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, _rw_timestamp ], 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, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + 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, _rw_timestamp ], 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 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -1386,21 +1386,21 @@ └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 4 ] } Table 0 - ├── columns: [ $expr2, $expr3, max($expr4), count, count filter(($expr5 < 10000:Int32)), count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count filter(($expr5 >= 1000000:Int32)), count(distinct $expr6), count(distinct $expr6) filter(($expr5 < 10000:Int32)), count(distinct $expr6) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr6) filter(($expr5 >= 1000000:Int32)), count(distinct $expr7), count(distinct $expr7) filter(($expr5 < 10000:Int32)), count(distinct $expr7) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr7) filter(($expr5 >= 1000000:Int32)) ] + ├── columns: [ $expr2, $expr3, max($expr4), count, count filter(($expr5 < 10000:Int32)), count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count filter(($expr5 >= 1000000:Int32)), count(distinct $expr6), count(distinct $expr6) filter(($expr5 < 10000:Int32)), count(distinct $expr6) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr6) filter(($expr5 >= 1000000:Int32)), count(distinct $expr7), count(distinct $expr7) filter(($expr5 < 10000:Int32)), count(distinct $expr7) filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count(distinct $expr7) filter(($expr5 >= 1000000:Int32)), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ $expr2, $expr3, $expr6, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8 ], 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 1 { columns: [ $expr2, $expr3, $expr6, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8, _rw_timestamp ], 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 2 { columns: [ $expr2, $expr3, $expr7, 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 2 { columns: [ $expr2, $expr3, $expr7, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, count_for_agg_call_12, _rw_timestamp ], 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, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + 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, _rw_timestamp ], 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 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -1459,17 +1459,22 @@ └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 2 ] } Table 0 - ├── columns: [ $expr2, $expr3, count, count filter(($expr4 < 10000:Int32)), count filter(($expr4 >= 10000:Int32) AND ($expr4 < 1000000:Int32)), count filter(($expr4 >= 1000000:Int32)), min($expr4), max($expr4), sum($expr4), count($expr4) ] + ├── columns: [ $expr2, $expr3, count, count filter(($expr4 < 10000:Int32)), count filter(($expr4 >= 10000:Int32) AND ($expr4 < 1000000:Int32)), count filter(($expr4 >= 1000000:Int32)), min($expr4), max($expr4), sum($expr4), count($expr4), _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 1 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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 } + Table 4294967294 + ├── columns: [ auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price, _rw_timestamp ] + ├── 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 eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -1525,13 +1530,13 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 2 ] } - Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id ], primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - Table 1 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -1595,13 +1600,13 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 2 ] } - Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id ], primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - Table 1 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 2 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id ], primary key: [ $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1, 0 ], read pk prefix len hint: 3 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction, _row_id], pk_columns: [bidder, auction, _row_id], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -1668,15 +1673,15 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 3 ] } - Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id, row_number ], primary key: [ $0 ASC, $2 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id, row_number, _rw_timestamp ], primary key: [ $0 ASC, $2 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id ], primary key: [ $0 ASC, $2 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr7, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 DESC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 3 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number, _rw_timestamp ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 2 } eowc_stream_error: 'Invalid input syntax: Only support window functions order by single column and in ascending order' - id: nexmark_q20 @@ -1756,20 +1761,20 @@ └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, _row_id, _rw_timestamp ], 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 1 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr7, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr7, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr7, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr7, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 - ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1, _rw_timestamp ] ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 14, 15 ] @@ -1821,11 +1826,11 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 1 ] } - Table 0 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id, _rw_timestamp ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - id: nexmark_q22 before: @@ -1863,11 +1868,16 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 1 ] } - Table 0 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 1 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 1 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _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 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _row_id, _rw_timestamp ] + ├── primary key: [ $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 7 ] + └── read pk prefix len hint: 1 eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -1964,22 +1974,22 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr4, max($expr5) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr4, max($expr5), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr4, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr4, max($expr5), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr4, max($expr5), count, _rw_timestamp ], 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, _row_id, $expr4 ] + ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr4, _rw_timestamp ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0, 3 ] @@ -2124,29 +2134,29 @@ StreamNoOp └── StreamExchange NoShuffle from 4 - Table 0 { columns: [ $expr2, $expr3, count($expr4) ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, count($expr4), _rw_timestamp ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr5 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr5, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ $expr2, $expr3, count($expr4), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ $expr2, $expr3, count($expr4), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr4, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr4, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr4, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr4, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 8 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 9 { columns: [ sum0(sum0(count)), sum0(count($expr4)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 { columns: [ sum0(sum0(count)), sum0(count($expr4)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 10 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ $expr4, count, _rw_timestamp ], 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 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -2243,21 +2253,21 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr4 ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr4, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr4, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr4, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -2354,21 +2364,21 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr4 ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr4, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr4, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 5 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr4, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -2469,25 +2479,31 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ $expr2, $expr3, count($expr4), $expr5 ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ $expr2, $expr3, count($expr4), $expr5, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ $expr2, $expr3, count($expr4), $expr5 ], primary key: [ $3 ASC, $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 3 } + Table 1 + ├── columns: [ $expr2, $expr3, count($expr4), $expr5, _rw_timestamp ] + ├── primary key: [ $3 ASC, $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 3 - Table 2 { columns: [ $expr2, $expr3, count($expr4), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ $expr2, $expr3, count($expr4), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 3 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr4, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr4, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr4, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr4, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 8 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - 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: 3 } + Table 4294967294 { columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 3 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -2596,29 +2612,29 @@ └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ min(max($expr5)), $expr6 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ min(max($expr5)), $expr6, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ min(min(max($expr5))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ min(min(max($expr5))), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { columns: [ $expr6, max($expr5), $expr2 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 { columns: [ $expr6, max($expr5), $expr2, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 3 { columns: [ $expr6, min(max($expr5)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ $expr6, min(max($expr5)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4 { columns: [ $expr2, max($expr5), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ $expr2, max($expr5), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ $expr2, $expr1, $expr3, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ $expr2, $expr1, $expr3, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr2, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr2, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ $expr4, $expr5, $expr1, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ $expr4, $expr5, $expr1, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ $expr4, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ $expr4, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ vnode, offset ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, offset, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 10 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 10 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ min_final ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ min_final, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml index 795941297290b..a6a2c284beb07 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 @@ -6,8 +6,8 @@ logical_plan: |- LogicalProject { exprs: [(t.x + t.y) as $expr4, t.y, 2:Int32, (1:Int32 + row_number) as $expr5, sum] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY $expr1 ORDER BY $expr2 ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), sum(t.x) OVER(PARTITION BY $expr3 ORDER BY $expr2 ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id, (t.y + t.y) as $expr1, (t.x - 1:Int32) as $expr2, (t.x * t.x) as $expr3] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp, (t.y + t.y) as $expr1, (t.x - 1:Int32) as $expr2, (t.x * t.x) as $expr3] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } - id: window function call in ORDER BY sql: | create table t(x int); @@ -16,8 +16,8 @@ LogicalProject { exprs: [t.x] } └─LogicalProject { exprs: [t.x, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalProject { exprs: [t.x, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } - id: window function call in GROUP BY sql: | create table t(x int); @@ -70,8 +70,8 @@ logical_plan: |- LogicalProject { exprs: [first_value] } └─LogicalOverWindow { window_functions: [first_value(t.x) OVER(ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING)] } - └─LogicalProject { exprs: [t.x, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalProject { exprs: [t.x, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } batch_error: |- Feature is not yet implemented: Window function with empty PARTITION BY is not supported because of potential bad performance. If you really need this, please workaround with something like `PARTITION BY 1::int`. Tracking issue: https://github.com/risingwavelabs/risingwave/issues/11505 @@ -85,8 +85,8 @@ logical_plan: |- LogicalProject { exprs: [first_value] } └─LogicalOverWindow { window_functions: [first_value(t.x) OVER(ROWS BETWEEN 2 FOLLOWING AND 2 FOLLOWING)] } - └─LogicalProject { exprs: [t.x, t._row_id, 2:Int32] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalProject { exprs: [t.x, t._row_id, t._rw_timestamp, 2:Int32] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } batch_error: |- Feature is not yet implemented: Window function with empty PARTITION BY is not supported because of potential bad performance. If you really need this, please workaround with something like `PARTITION BY 1::int`. Tracking issue: https://github.com/risingwavelabs/risingwave/issues/11505 @@ -110,8 +110,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, first_value] } └─LogicalOverWindow { window_functions: [first_value(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [first_value(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING)] } @@ -130,8 +130,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, first_value] } └─LogicalOverWindow { window_functions: [first_value(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [first_value(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN 1 PRECEDING AND 1 PRECEDING)] } @@ -151,8 +151,8 @@ logical_plan: |- LogicalProject { exprs: [sum] } └─LogicalOverWindow { window_functions: [sum(t.x) OVER(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)] } - └─LogicalProject { exprs: [t.x, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalProject { exprs: [t.x, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } batch_plan: |- BatchNestedLoopJoin { type: Inner, predicate: true, output: [sum(sum(t.x))] } ├─BatchExchange { order: [], dist: Single } @@ -173,8 +173,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, sum, max, min] } └─LogicalOverWindow { window_functions: [sum(t.x) OVER(PARTITION BY t.y ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), max(t.x) OVER(PARTITION BY t.y ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), min(t.w) OVER(PARTITION BY t.y ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w)] } @@ -202,8 +202,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, min] } └─LogicalOverWindow { window_functions: [min(t.x) OVER(PARTITION BY t.y ROWS BETWEEN 10 PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [min(t.x) OVER(PARTITION BY t.y ROWS BETWEEN 10 PRECEDING AND CURRENT ROW)] } @@ -222,8 +222,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, min] } └─LogicalOverWindow { window_functions: [min(t.x) OVER(PARTITION BY t.y ROWS BETWEEN 1 PRECEDING AND 2 FOLLOWING)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [min(t.x) OVER(PARTITION BY t.y ROWS BETWEEN 1 PRECEDING AND 2 FOLLOWING)] } @@ -262,8 +262,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, max] } └─LogicalOverWindow { window_functions: [max(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC RANGE BETWEEN 100 PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [max(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC RANGE BETWEEN 100 PRECEDING AND CURRENT ROW)] } @@ -282,8 +282,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, max] } └─LogicalOverWindow { window_functions: [max(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC RANGE BETWEEN 100 PRECEDING AND UNBOUNDED FOLLOWING)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [max(t.x) OVER(PARTITION BY t.y ORDER BY t.x ASC RANGE BETWEEN 100 PRECEDING AND UNBOUNDED FOLLOWING)] } @@ -302,8 +302,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, first_value] } └─LogicalOverWindow { window_functions: [first_value(t.x) OVER(PARTITION BY t.y ORDER BY t.x DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [first_value(t.x) OVER(PARTITION BY t.y ORDER BY t.x DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -327,8 +327,8 @@ └─LogicalFilter { predicate: (t.z > 0:Int32) AND (t.y > 0:Int32) AND (t.x > 0:Int32) AND ($expr4 <= 3.0:Decimal) AND ($expr5 > 1.0:Decimal) } └─LogicalProject { exprs: [t.x, t.y, t.z, Case((count = 0:Int32), null:Decimal, Sqrt((Greatest((sum::Decimal - ((sum::Decimal * sum::Decimal) / count::Decimal)), 0:Int32::Decimal) / count::Decimal))) as $expr4, Case((count <= 1:Int32), null:Decimal, Sqrt((Greatest((sum::Decimal - ((sum::Decimal * sum::Decimal) / count::Decimal)), 0:Int32::Decimal) / (count - 1:Int32)::Decimal))) as $expr5] } └─LogicalOverWindow { window_functions: [sum($expr1) OVER(PARTITION BY t.z ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), sum($expr2) OVER(PARTITION BY t.z ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), count($expr2) OVER(PARTITION BY t.z ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), sum($expr3) OVER(PARTITION BY t.z ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), sum(t.x) OVER(PARTITION BY t.z ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), count(t.x) OVER(PARTITION BY t.z ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id, ((t.x - t.y) * (t.x - t.y)) as $expr1, (t.x - t.y) as $expr2, (t.x * t.x) as $expr3] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp, ((t.x - t.y) * (t.x - t.y)) as $expr1, (t.x - t.y) as $expr2, (t.x * t.x) as $expr3] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.x, t.y, t.z, Case((count = 0:Int32), null:Decimal, Sqrt((Greatest((sum::Decimal - ((sum::Decimal * sum::Decimal) / count::Decimal)), 0:Decimal) / count::Decimal))) as $expr4, Case((count <= 1:Int32), null:Decimal, Sqrt((Greatest((sum::Decimal - ((sum::Decimal * sum::Decimal) / count::Decimal)), 0:Decimal) / (count - 1:Int32)::Decimal))) as $expr5] } @@ -362,8 +362,8 @@ └─LogicalFilter { predicate: (t.z > 0:Int32) AND (t.y > 0:Int32) AND (t.x > 0:Int32) AND ($expr4 <= 3.0:Decimal) } └─LogicalProject { exprs: [t.x, t.y, t.z, (sum::Decimal / count::Decimal) as $expr4] } └─LogicalOverWindow { window_functions: [sum($expr1) OVER(PARTITION BY $expr2 ORDER BY $expr3 ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), count($expr1) OVER(PARTITION BY $expr2 ORDER BY $expr3 ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id, (t.z * t.z) as $expr1, (t.y + 1:Int32) as $expr2, Abs(t.w) as $expr3] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp, (t.z * t.z) as $expr1, (t.y + 1:Int32) as $expr2, Abs(t.w) as $expr3] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t.w, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.x, t.y, t.z, (sum::Decimal / count::Decimal) as $expr4] } └─LogicalFilter { predicate: (t.z > 0:Int32) AND (t.y > 0:Int32) AND (t.x > 0:Int32) AND ((sum::Decimal / count::Decimal) <= 3.0:Decimal) } @@ -399,8 +399,8 @@ logical_plan: |- LogicalProject { exprs: [row_number, rank, dense_rank] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), dense_rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalProject { exprs: [t.x, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_stream: |- LogicalProject { exprs: [row_number, rank, dense_rank] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), dense_rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -425,8 +425,8 @@ logical_plan: |- LogicalProject { exprs: [row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [row_number] } @@ -451,8 +451,8 @@ └─LogicalFilter { predicate: (row_number < 3:Int32) } └─LogicalProject { exprs: [t.x, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalProject { exprs: [t.x, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -477,8 +477,8 @@ └─LogicalFilter { predicate: (row_number <= 3:Int32) } └─LogicalProject { exprs: [t.x, t.y, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [t.x ASC], limit: 3, offset: 0, group_key: [t.y] } └─LogicalScan { table: t, columns: [t.x, t.y] } @@ -503,8 +503,8 @@ └─LogicalFilter { predicate: (row_number < 3:Int32) AND (t.x > t.y) } └─LogicalProject { exprs: [t.x, t.y, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: (t.x > t.y) } └─LogicalTopN { order: [t.x ASC], limit: 2, offset: 0, group_key: [t.y] } @@ -532,8 +532,8 @@ └─LogicalFilter { predicate: (rank <= 3:Int32) } └─LogicalProject { exprs: [t.x, t.y, rank] } └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [t.x ASC], limit: 3, offset: 0, with_ties: true, group_key: [t.y] } └─LogicalScan { table: t, columns: [t.x, t.y] } @@ -559,8 +559,8 @@ └─LogicalFilter { predicate: (rank <= 3:Int32) AND (rank > 1:Int32) } └─LogicalProject { exprs: [t.x, t.y, rank] } └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.x, t.y] } @@ -658,8 +658,8 @@ └─LogicalFilter { predicate: (3:Int32 < row_number) AND (row_number = 6:Int32) AND (row_number <= 5:Int32) } └─LogicalProject { exprs: [t.x, t.y, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.x, t.y] } @@ -686,8 +686,8 @@ └─LogicalFilter { predicate: (row_number = 1:Int32) } └─LogicalProject { exprs: [t.x, t.y, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [t.y ASC], limit: 1, offset: 0, group_key: [t.x] } └─LogicalScan { table: t, columns: [t.x, t.y] } @@ -713,8 +713,8 @@ └─LogicalFilter { predicate: (row_number <= 1:Int32) } └─LogicalProject { exprs: [t.x, t.y, t.z, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x, t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.z] } └─LogicalTopN { order: [t.x ASC], limit: 1, offset: 0, group_key: [t.x, t.y] } @@ -746,21 +746,21 @@ └── BatchPlanNode Table 0 - ├── columns: [ t_x, t_y, t_z, t__row_id ] + ├── columns: [ t_x, t_y, t_z, t__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $0 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ z, t.x, t.y ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ z, t.x, t.y, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } - id: TopN among multiple window function calls sql: | @@ -780,8 +780,8 @@ └─LogicalFilter { predicate: (row_number < 10:Int32) AND (row_number < 10:Int32) AND (rank < 10:Int32) } └─LogicalProject { exprs: [t.x, t.y, t.z, row_number, row_number, rank] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), row_number() OVER(PARTITION BY t.x, t.y ORDER BY t.z ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), rank() OVER(PARTITION BY t.x, t.y ORDER BY t.z ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_stream: |- LogicalProject { exprs: [row_number] } └─LogicalFilter { predicate: (row_number < 10:Int32) AND (row_number < 10:Int32) } @@ -829,8 +829,8 @@ └─LogicalFilter { predicate: (row_number < 10:Int32) } └─LogicalProject { exprs: [t.x, t.y, t.z, row_number, row_number, rank] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), row_number() OVER(PARTITION BY t.x, t.y ORDER BY t.z ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), rank() OVER(PARTITION BY t.x, t.y ORDER BY t.z ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_stream: |- LogicalProject { exprs: [row_number, rank] } └─LogicalFilter { predicate: (row_number < 10:Int32) } @@ -942,8 +942,8 @@ └─LogicalFilter { predicate: (row_number <= 3:Int32) } └─LogicalProject { exprs: [t.x, t.y, t.z, row_number] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.z] } @@ -976,9 +976,9 @@ │ └─LogicalFilter { predicate: (row_number = 1:Int32) } │ └─LogicalProject { exprs: [t.x, t.y, t.z, row_number] } │ └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - │ └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id] } - │ └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id] } + │ └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } + │ └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t.x, t.z, t.y, t.z] } @@ -1003,8 +1003,8 @@ logical_plan: |- LogicalProject { exprs: [t.x, t.y, t.z, rank, sum, row_number, rank] } └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), sum(t.x) OVER(PARTITION BY t.z ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id] } - └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id] } + └─LogicalProject { exprs: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.x, t.y, t.z, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } @@ -1143,8 +1143,8 @@ logical_plan: |- LogicalProject { exprs: [count, count, count, count, count, count] } └─LogicalOverWindow { window_functions: [count() OVER(PARTITION BY 1:Int32 ORDER BY t.i ASC RANGE BETWEEN 1 PRECEDING AND CURRENT ROW), count() OVER(PARTITION BY 1:Int32 ORDER BY t.bi ASC RANGE BETWEEN 1 PRECEDING AND CURRENT ROW), count() OVER(PARTITION BY 1:Int32 ORDER BY t.d ASC RANGE BETWEEN 1.5 PRECEDING AND CURRENT ROW), count() OVER(PARTITION BY 1:Int32 ORDER BY t.f ASC RANGE BETWEEN 1.5 PRECEDING AND CURRENT ROW), count() OVER(PARTITION BY 1:Int32 ORDER BY t.ts ASC RANGE BETWEEN 1 day 01:00:00 PRECEDING AND CURRENT ROW), count() OVER(PARTITION BY 1:Int32 ORDER BY t.tstz ASC RANGE BETWEEN 00:01:00 PRECEDING AND CURRENT ROW)] } - └─LogicalProject { exprs: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id, 1:Int32] } - └─LogicalScan { table: t, columns: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id] } + └─LogicalProject { exprs: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id, t._rw_timestamp, 1:Int32] } + └─LogicalScan { table: t, columns: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_stream: |- LogicalProject { exprs: [count, count, count, count, count, count] } └─LogicalOverWindow { window_functions: [count() OVER(PARTITION BY 1:Int32 ORDER BY t.tstz ASC RANGE BETWEEN 00:01:00 PRECEDING AND CURRENT ROW)] } @@ -1248,8 +1248,8 @@ logical_plan: |- LogicalProject { exprs: [first_value, first_value, first_value, first_value, first_value, first_value] } └─LogicalOverWindow { window_functions: [first_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.i ASC SESSION WITH GAP 10), first_value(t.bi) OVER(PARTITION BY t.i ORDER BY t.bi ASC SESSION WITH GAP 10), first_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.d ASC SESSION WITH GAP 1.5), first_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.f ASC SESSION WITH GAP 1.5), first_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.ts ASC SESSION WITH GAP 1 day 01:00:00), first_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.tstz ASC SESSION WITH GAP 00:01:00)] } - └─LogicalProject { exprs: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id] } - └─LogicalScan { table: t, columns: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id] } + └─LogicalProject { exprs: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.i, t.bi, t.d, t.f, t.da, t.t, t.ts, t.tstz, t.itv, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_stream: |- LogicalProject { exprs: [first_value, first_value, first_value, first_value, first_value, first_value] } └─LogicalOverWindow { window_functions: [first_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.tstz ASC SESSION WITH GAP 00:01:00)] } @@ -1296,8 +1296,8 @@ logical_plan: |- LogicalProject { exprs: [first_value, last_value] } └─LogicalOverWindow { window_functions: [first_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.ts ASC SESSION WITH GAP 00:10:00), last_value(t.i) OVER(PARTITION BY t.bi ORDER BY t.ts ASC SESSION WITH GAP 00:10:00)] } - └─LogicalProject { exprs: [t.i, t.bi, t.ts, t._row_id] } - └─LogicalScan { table: t, columns: [t.i, t.bi, t.ts, t._row_id] } + └─LogicalProject { exprs: [t.i, t.bi, t.ts, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.i, t.bi, t.ts, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [first_value, last_value] } diff --git a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml index 2d033d7724b46..376d185d06eda 100644 --- a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml @@ -8,7 +8,7 @@ └─LogicalFilter { predicate: false:Boolean } └─LogicalAgg { aggs: [min(t.v1)] } └─LogicalProject { exprs: [t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: 'LogicalValues { rows: [], schema: Schema { fields: [min(t.v1):Int32] } }' - name: filter should not transpose limit sql: | @@ -20,7 +20,7 @@ └─LogicalShare { id: 3 } └─LogicalLimit { limit: 10, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) } └─LogicalLimit { limit: 10, offset: 0 } @@ -35,7 +35,7 @@ └─LogicalShare { id: 3 } └─LogicalTopN { order: [t.v1 ASC], limit: 10, offset: 0 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) } └─LogicalTopN { order: [t.v1 ASC], limit: 10, offset: 0 } @@ -55,7 +55,7 @@ └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, t.ts, window_start, window_end] } └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t.ts) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: (window_start >= '1997-07-02':Date) AND (window_end >= '1997-07-03':Date) AND (window_start >= (t.ts + '1 day':Interval)) AND (window_end > (t.ts + '4 days':Interval)) } └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } @@ -72,7 +72,7 @@ └─LogicalProject { exprs: [window_end, t.v4, t.v2] } └─LogicalHopWindow { time_col: t.ts, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t.ts) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [window_end, t.v4, t.v2] } └─LogicalFilter { predicate: (window_end > '2022-01-01':Date) } @@ -89,9 +89,9 @@ └─LogicalShare { id: 4 } └─LogicalUnion { all: true } ├─LogicalProject { exprs: [t1.v1, t1.v2, t1.v3] } - │ └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id] } + │ └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.v1, t2.v2, t2.v3] } - └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2.v3, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalUnion { all: true } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1.v3], predicate: (t1.v1 = 10:Int32) AND (t1.v2 = 20:Int32) AND (t1.v3 = 30:Int32) } @@ -105,7 +105,7 @@ └─LogicalFilter { predicate: ($expr1 > 10:Int32) AND (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND ($expr1 > $expr2) AND ($expr1 > t.v1) } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr1, (t.v3 * t.v4) as $expr2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr1, (t.v3 * t.v4) as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4], predicate: ((t.v1 + t.v2) > 10:Int32) AND (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND ((t.v1 + t.v2) > (t.v3 * t.v4)) AND ((t.v1 + t.v2) > t.v1) } @@ -120,7 +120,7 @@ └─LogicalProject { exprs: [t.v1, t.v2, t.v3, count, count(1:Int32)] } └─LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count, count(1:Int32)] } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, 1:Int32] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: (count > t.v1) AND (count > count(1:Int32)) } └─LogicalAgg { group_key: [t.v1, t.v2, t.v3], aggs: [count, count(1:Int32)] } @@ -136,7 +136,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($3)] } └─LogicalProjectSet { select_list: [$0, $1, $2, Unnest($3)] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.arr, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.arr, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.v1, t.v2, t.v3, Unnest($3)] } └─LogicalFilter { predicate: (Unnest($3) = 30:Int32) } @@ -153,8 +153,8 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNull(t1.v1) AND IsNull(t1.v2) } @@ -170,8 +170,8 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: LeftOuter, on: true, output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: IsNull(t2.v3) AND IsNull(t2.v4) } └─LogicalJoin { type: LeftOuter, on: true, output: all } @@ -188,8 +188,8 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: RightOuter, on: true, output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) } └─LogicalJoin { type: LeftOuter, on: true, output: [t1.v1, t1.v2, t2.v3, t2.v4] } @@ -206,8 +206,8 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t1.v1, t1.v2, t2.v3, t2.v4] } └─LogicalJoin { type: FullOuter, on: true, output: all } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalFilter { predicate: IsNull(t1.v1) AND IsNull(t1.v2) AND IsNull(t2.v3) AND IsNull(t2.v4) } └─LogicalJoin { type: FullOuter, on: true, output: all } @@ -224,9 +224,9 @@ └─LogicalShare { id: 4 } └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.v1, t2.v2] } - └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNull(t1.v1) AND IsNull(t1.v2) } @@ -242,9 +242,9 @@ └─LogicalShare { id: 4 } └─LogicalProject { exprs: [t1.v1, t1.v2] } └─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.v1, t2.v2] } - └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftAnti, on: true, output: all } ├─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: IsNull(t1.v1) AND IsNull(t1.v2) } diff --git a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml index e627824ca70dc..e175558038c81 100644 --- a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml @@ -503,7 +503,7 @@ logical_plan: |- LogicalProject { exprs: [t.v] } └─LogicalFilter { predicate: (t.k = 2147483648:Int64) } - └─LogicalScan { table: t, columns: [t.k, t.v] } + └─LogicalScan { table: t, columns: [t.k, t.v, t._rw_timestamp] } batch_plan: 'BatchValues { rows: [] }' - name: When OR clauses contain non-overlapping conditions,, we can pushdown serveral scan_range. before: diff --git a/src/frontend/planner_test/tests/testdata/output/rw_timestamp.yaml b/src/frontend/planner_test/tests/testdata/output/rw_timestamp.yaml new file mode 100644 index 0000000000000..d61d7f3e8802d --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/rw_timestamp.yaml @@ -0,0 +1,41 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- sql: | + create table t (a int); + select _rw_timestamp from t; + logical_plan: |- + LogicalProject { exprs: [t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchScan { table: t, columns: [t._rw_timestamp], distribution: SomeShard } + stream_error: |- + Not supported: selecting `_rw_timestamp` in a streaming query is not allowed + HINT: please run the sql in batch mode or remove the column `_rw_timestamp` from the streaming query +- sql: | + create table t (a int); + select t.*, _rw_timestamp from t; + logical_plan: |- + LogicalProject { exprs: [t.a, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchScan { table: t, columns: [t.a, t._rw_timestamp], distribution: SomeShard } + stream_error: |- + Not supported: selecting `_rw_timestamp` in a streaming query is not allowed + HINT: please run the sql in batch mode or remove the column `_rw_timestamp` from the streaming query +- sql: | + create table t (a int); + create index idx on t(a); + select _rw_timestamp from t where a = 1; + logical_plan: |- + LogicalProject { exprs: [t._rw_timestamp] } + └─LogicalFilter { predicate: (t.a = 1:Int32) } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchLookupJoin { type: Inner, predicate: idx.t._row_id IS NOT DISTINCT FROM t._row_id, output: [t._rw_timestamp], lookup table: t } + └─BatchExchange { order: [], dist: UpstreamHashShard(idx.t._row_id) } + └─BatchScan { table: idx, columns: [idx.t._row_id], scan_ranges: [idx.a = Int32(1)], distribution: SomeShard } + stream_error: |- + Not supported: selecting `_rw_timestamp` in a streaming query is not allowed + HINT: please run the sql in batch mode or remove the column `_rw_timestamp` from the streaming query diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index abdb5f29a55a8..c6e38646cfe2f 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -266,27 +266,27 @@ StreamNoOp └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ sum0(count), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum0(count), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ t_a ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ t_a, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ t_a, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ t_a, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ t_a ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ t_a, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ t_a, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ t_a, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ t_a, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ t_a, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ count ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ count, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: self_join_with_temporal_filter_one_side before: diff --git a/src/frontend/planner_test/tests/testdata/output/shared_views.yaml b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml index 2a710d3602a02..88c034e10a15e 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml @@ -11,16 +11,16 @@ ├─LogicalShare { id: 3 } │ └─LogicalProject { exprs: [(t1.x + t1.y) as $expr1] } │ └─LogicalFilter { predicate: (t1.y > 0:Int32) } - │ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + │ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalShare { id: 7 } └─LogicalProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3] } └─LogicalFilter { predicate: ($expr1 = t1.x) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalShare { id: 3 } └─LogicalProject { exprs: [(t1.x + t1.y) as $expr1] } └─LogicalFilter { predicate: (t1.y > 0:Int32) } - └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden)], stream_key: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard($expr1, t1._row_id, t1._row_id, t1._row_id, t1.x) } diff --git a/src/frontend/planner_test/tests/testdata/output/short_circuit.yaml b/src/frontend/planner_test/tests/testdata/output/short_circuit.yaml index 47f1c73897a25..5c3fb5218fd05 100644 --- a/src/frontend/planner_test/tests/testdata/output/short_circuit.yaml +++ b/src/frontend/planner_test/tests/testdata/output/short_circuit.yaml @@ -27,7 +27,7 @@ select true or 'abc'::int > c1 from t1; logical_plan: |- LogicalProject { exprs: [(true:Boolean OR ('abc':Varchar::Int32 > t1.c1)) as $expr1] } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [true:Boolean] } @@ -39,7 +39,7 @@ select false and 'abc'::int > c1 from t1; logical_plan: |- LogicalProject { exprs: [(false:Boolean AND ('abc':Varchar::Int32 > t1.c1)) as $expr1] } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [false:Boolean] } diff --git a/src/frontend/planner_test/tests/testdata/output/sink.yaml b/src/frontend/planner_test/tests/testdata/output/sink.yaml index f914fb225c9fd..0bc336bd21a26 100644 --- a/src/frontend/planner_test/tests/testdata/output/sink.yaml +++ b/src/frontend/planner_test/tests/testdata/output/sink.yaml @@ -62,7 +62,8 @@ │ ├── v2: double precision │ ├── v3: character varying │ ├── v5: numeric - │ └── t1.v4: bigint + │ ├── t1.v4: bigint + │ └── _rw_timestamp: timestamp with time zone ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 4, 5 ] @@ -70,7 +71,7 @@ └── vnode column idx: 2 Table 1 - ├── columns: [ vnode: smallint, v3: character varying, v4: bigint, backfill_finished: boolean, row_count: bigint ] + ├── columns: [ vnode: smallint, v3: character varying, v4: bigint, backfill_finished: boolean, row_count: bigint, _rw_timestamp: timestamp with time zone ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4 ] ├── distribution key: [ 0 ] @@ -113,7 +114,8 @@ │ ├── v1: integer │ ├── v2: double precision │ ├── v3: character varying - │ └── v5: numeric + │ ├── v5: numeric + │ └── _rw_timestamp: timestamp with time zone ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 6, 7 ] @@ -121,7 +123,7 @@ └── vnode column idx: 2 Table 1 - ├── columns: [ vnode: smallint, v1: integer, v2: double precision, backfill_finished: boolean, row_count: bigint ] + ├── columns: [ vnode: smallint, v1: integer, v2: double precision, backfill_finished: boolean, row_count: bigint, _rw_timestamp: timestamp with time zone ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4 ] ├── distribution key: [ 0 ] diff --git a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml index 48caec86bd940..7a0ac41a4b85b 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,19 +30,19 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ s_v, s_t__row_id ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ s_v, s_t__row_id, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ max(s_v), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ max(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: sum_on_single before: @@ -68,17 +68,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(s_v), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: cnt_on_single before: @@ -104,17 +104,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ count(s_v), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ count(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: string_agg_on_single before: @@ -143,19 +143,29 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ s_v, s_t__row_id, s_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ s_v, s_t__row_id, s_s, ',':Varchar, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 1 { columns: [ string_agg(s_s, ',':Varchar order_by(s_v ASC)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ string_agg(s_s, ',':Varchar order_by(s_v ASC)), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 2 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: extreme_on_T before: @@ -192,29 +202,35 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ max(t_v), _vnode ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ max(t_v), _vnode, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ max(max(t_v)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ max(max(t_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ _vnode, t_v, t__row_id ] + ├── columns: [ _vnode, t_v, t__row_id, _rw_timestamp ] ├── 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 3 { columns: [ _vnode, max(t_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 + ├── columns: [ _vnode, max(t_v), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 Table 4 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: extreme_on_AO before: @@ -244,17 +260,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ max(max(ao_v)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ max(max(ao_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: sum_on_T before: @@ -289,17 +305,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum(t_v)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum(t_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: sum_on_AO before: @@ -329,17 +345,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum(ao_v)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum(ao_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: cnt_on_T before: @@ -374,17 +390,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum0(count(t_v)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum0(count(t_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: cnt_on_AO before: @@ -414,17 +430,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum0(count(ao_v)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum0(count(ao_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: string_agg_on_T before: @@ -459,19 +475,29 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_o, t__row_id, t_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ t_o, t__row_id, t_s, ',':Varchar, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 1 { columns: [ string_agg(t_s, ',':Varchar order_by(t_o ASC)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ string_agg(t_s, ',':Varchar order_by(t_o ASC)), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: string_agg_on_AO before: @@ -501,19 +527,29 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ao_o, ao__row_id, ao_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ ao_o, ao__row_id, ao_s, ',':Varchar, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 1 { columns: [ string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: extreme_count_on_T before: @@ -550,12 +586,12 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ max(t_v), _vnode ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ max(t_v), _vnode, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ max(max(t_v)), sum0(count(t_v)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ max(max(t_v)), sum0(count(t_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ _vnode, t_v, t__row_id ] + ├── columns: [ _vnode, t_v, t__row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] ├── value indices: [ 1, 2 ] ├── distribution key: [ 2 ] @@ -563,7 +599,7 @@ └── vnode column idx: 0 Table 3 - ├── columns: [ _vnode, max(t_v), count(t_v), count ] + ├── columns: [ _vnode, max(t_v), count(t_v), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [] @@ -571,14 +607,14 @@ └── vnode column idx: 0 Table 4 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a2 ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, a2, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - id: extreme_count_on_AO before: @@ -608,17 +644,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ max(max(ao_v)), sum0(count(ao_v)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ max(max(ao_v)), sum0(count(ao_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a2 ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, a2, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - id: count_string_agg_on_T before: @@ -653,24 +689,29 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_o, t__row_id, t_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ t_o, t__row_id, t_s, ',':Varchar, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 1 - ├── columns: [ count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count ] + ├── columns: [ count(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count, _rw_timestamp ] ├── primary key: [] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [] └── read pk prefix len hint: 0 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a2 ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, a2, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - id: count_string_agg_on_AO before: @@ -700,19 +741,24 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ao_o, ao__row_id, ao_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ ao_o, ao__row_id, ao_s, ',':Varchar, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ count(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a2 ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, a2, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - id: extreme_string_agg_on_T before: @@ -747,21 +793,31 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_v, t__row_id ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ t_v, t__row_id, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ t_o, t__row_id, t_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ t_o, t__row_id, t_s, ',':Varchar, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 2 { columns: [ max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 2 + ├── columns: [ max(t_v), string_agg(t_s, ',':Varchar order_by(t_o ASC)), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 3 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a2 ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, a2, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - id: extreme_string_agg_on_AO before: @@ -791,19 +847,24 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ao_o, ao__row_id, ao_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ ao_o, ao__row_id, ao_s, ',':Varchar, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 + ├── columns: [ max(ao_v), string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count, _rw_timestamp ] + ├── primary key: [] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, a2 ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ a1, a2, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - id: extreme_on_T_by_k before: @@ -837,19 +898,24 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_k, t_v, t__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ t_k, t_v, t__row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ t_k, max(t_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ t_k, max(t_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, t.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, t.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: extreme_on_Tk_by_k before: @@ -876,19 +942,19 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ tk_k, tk_v, tk_t__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ tk_k, tk_v, tk_t__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ tk_k, max(tk_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ tk_k, max(tk_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 - ├── columns: [ vnode, k, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, k, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1, tk.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, tk.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: extreme_on_S_by_k before: @@ -922,19 +988,24 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ s_k, s_v, s_t__row_id ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ s_k, s_v, s_t__row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ s_k, max(s_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ s_k, max(s_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1, s.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, s.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: extreme_on_AO_by_k before: @@ -962,17 +1033,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ao_k, max(ao_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ ao_k, max(ao_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, ao.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, ao.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: sum_on_T_by_k before: @@ -1005,17 +1076,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_k, sum(t_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ t_k, sum(t_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, t.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, t.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: sum_on_Tk_by_k before: @@ -1042,17 +1113,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ tk_k, sum(tk_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ tk_k, sum(tk_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, k, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, k, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1, tk.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, tk.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: sum_on_S_by_k before: @@ -1085,17 +1156,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ s_k, sum(s_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ s_k, sum(s_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1, s.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, s.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: sum_on_AO_by_k before: @@ -1123,17 +1194,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ao_k, sum(ao_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ ao_k, sum(ao_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, ao.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, ao.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: cnt_on_T_by_k before: @@ -1166,17 +1237,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t_k, count(t_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ t_k, count(t_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, t.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, t.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: cnt_on_Tk_by_k before: @@ -1203,17 +1274,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ tk_k, count(tk_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ tk_k, count(tk_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, k, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, k, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1, tk.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, tk.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: cnt_on_S_by_k before: @@ -1246,17 +1317,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ s_k, count(s_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ s_k, count(s_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1, s.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, s.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: cnt_on_AO_by_k before: @@ -1284,17 +1355,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ao_k, count(ao_v), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ ao_k, count(ao_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, ao.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, ao.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: string_agg_on_T_by_k before: @@ -1332,28 +1403,28 @@ └── BatchPlanNode Table 0 - ├── columns: [ t_k, t_o, t__row_id, t_s, ',':Varchar ] + ├── columns: [ t_k, t_o, t__row_id, t_s, ',':Varchar, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 1, 2, 3, 4 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ t_k, string_agg(t_s, ',':Varchar order_by(t_o ASC)), count ] + ├── columns: [ t_k, string_agg(t_s, ',':Varchar order_by(t_o ASC)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, t.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, t.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: string_agg_on_Tk_by_k before: @@ -1383,13 +1454,24 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ tk_k, tk_o, tk_t__row_id, tk_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ tk_k, tk_o, tk_t__row_id, tk_s, ',':Varchar, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 1, 2, 3, 4 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ tk_k, string_agg(tk_s, ',':Varchar order_by(tk_o ASC)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ tk_k, string_agg(tk_s, ',':Varchar order_by(tk_o ASC)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ vnode, k, t__row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 + ├── columns: [ vnode, k, t__row_id, backfill_finished, row_count, _rw_timestamp ] + ├── 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: [ a1, tk.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, tk.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: string_agg_on_S_by_k before: @@ -1427,28 +1509,28 @@ └── BatchPlanNode Table 0 - ├── columns: [ s_k, s_o, s_t__row_id, s_s, ',':Varchar ] + ├── columns: [ s_k, s_o, s_t__row_id, s_s, ',':Varchar, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 1, 2, 3, 4 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ s_k, string_agg(s_s, ',':Varchar order_by(s_o ASC)), count ] + ├── columns: [ s_k, string_agg(s_s, ',':Varchar order_by(s_o ASC)), count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 2 - ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count ] + ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a1, s.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, s.k, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: string_agg_on_AO_by_k before: @@ -1478,17 +1560,27 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ ao_k, ao_o, ao__row_id, ao_s, ',':Varchar ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ ao_k, ao_o, ao__row_id, ao_s, ',':Varchar, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 1, 2, 3, 4 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ ao_k, string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 + ├── columns: [ ao_k, string_agg(ao_s, ',':Varchar order_by(ao_o ASC)), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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, ao.k ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ a1, ao.k, _rw_timestamp ], 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/struct_field_access.yaml b/src/frontend/planner_test/tests/testdata/output/struct_field_access.yaml index aaa620dc56ed2..19b77a9287169 100644 --- a/src/frontend/planner_test/tests/testdata/output/struct_field_access.yaml +++ b/src/frontend/planner_test/tests/testdata/output/struct_field_access.yaml @@ -12,39 +12,39 @@ select (t.c).x from t logical_plan: |- LogicalProject { exprs: [Field(t.c, 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } - sql: | create table t(c STRUCT); select (c).x from t logical_plan: |- LogicalProject { exprs: [Field(t.c, 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } - sql: | create table t(c STRUCT); -- select (t).c.x from t select (t.c).x from t logical_plan: |- LogicalProject { exprs: [Field(t.c, 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } - sql: | create table t(c STRUCT); select (t.c).* from t logical_plan: |- LogicalProject { exprs: [Field(t.c, 0:Int32) as $expr1, Field(t.c, 1:Int32) as $expr2] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } - sql: | create table t(c STRUCT); select (c).* from t logical_plan: |- LogicalProject { exprs: [Field(t.c, 0:Int32) as $expr1, Field(t.c, 1:Int32) as $expr2] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } - sql: | create table t(c STRUCT); -- select (t).c.* from t select (t.c).* from t logical_plan: |- LogicalProject { exprs: [Field(t.c, 0:Int32) as $expr1, Field(t.c, 1:Int32) as $expr2] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } - sql: | create table t1(c STRUCT); create table t2(c STRUCT); @@ -63,24 +63,24 @@ logical_plan: |- LogicalProject { exprs: [t1.c] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.c, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.c, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.c] } └─LogicalFilter { predicate: (Field(t2.c, 1:Int32) = 100:Int32) AND (Field(CorrelatedInputRef { index: 0, correlated_id: 1 }, 0:Int32) = Field(t2.c, 0:Int32)) AND (Field(t2.c, 0:Int32) = 1000:Int32) AND (Field(CorrelatedInputRef { index: 0, correlated_id: 1 }, 1:Int32) = Field(t2.c, 1:Int32)) } - └─LogicalScan { table: t2, columns: [t2.c, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.c, t2._row_id, t2._rw_timestamp] } - sql: | create schema s; create table s.t(a STRUCT); select s.t.a from s.t; logical_plan: |- LogicalProject { exprs: [t.a] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - sql: | create schema s; create table s.t(a STRUCT); select (s.t.a).b from s.t; logical_plan: |- LogicalProject { exprs: [Field(t.a, 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - sql: | create schema s; create table s.t(a STRUCT); @@ -88,7 +88,7 @@ select (s.t.a).b from s.t; logical_plan: |- LogicalProject { exprs: [Field(t.a, 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - sql: | create schema s; create table s.t(a STRUCT); @@ -96,7 +96,7 @@ select (s.t.a).b from s.t; logical_plan: |- LogicalProject { exprs: [Field(t.a, 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - sql: | create schema s; create table s.t(a STRUCT); @@ -104,7 +104,7 @@ select (s.t.a).* from s.t; logical_plan: |- LogicalProject { exprs: [Field(t.a, 0:Int32) as $expr1, Field(t.a, 1:Int32) as $expr2] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - sql: | create schema s; create table s.t(a STRUCT); @@ -112,30 +112,30 @@ select (s.t.a).* from s.t; logical_plan: |- LogicalProject { exprs: [Field(t.a, 0:Int32) as $expr1, Field(t.a, 1:Int32) as $expr2] } - └─LogicalScan { table: t, columns: [t.a, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } - sql: | create schema t; create table t.t(t STRUCT); select (t.t.t).t from t.t; logical_plan: |- LogicalProject { exprs: [Field(t.t, 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.t, t._row_id] } + └─LogicalScan { table: t, columns: [t.t, t._row_id, t._rw_timestamp] } - sql: | create schema t; create table t.t(t STRUCT>); select (t.t.t).t.t from t.t; logical_plan: |- LogicalProject { exprs: [Field(Field(t.t, 0:Int32), 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.t, t._row_id] } + └─LogicalScan { table: t, columns: [t.t, t._row_id, t._rw_timestamp] } - sql: | create table t(c STRUCT[]); select (c[1]).x from t logical_plan: |- LogicalProject { exprs: [Field(ArrayAccess(t.c, 1:Int32), 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } - sql: | create table t(c STRUCT); select (case when true then c end).x from t logical_plan: |- LogicalProject { exprs: [Field(Case(true:Boolean, t.c), 0:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.c, t._row_id] } + └─LogicalScan { table: t, columns: [t.c, t._row_id, t._rw_timestamp] } diff --git a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml index 3e48801b61425..907aa209c6d2c 100644 --- a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml @@ -36,7 +36,7 @@ select (t.country).city,t.country,(country).city.address from t; logical_plan: |- LogicalProject { exprs: [Field(t.country, 1:Int32) as $expr1, t.country, Field(Field(t.country, 1:Int32), 0:Int32) as $expr2] } - └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } + └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } create_source: format: plain encode: protobuf @@ -73,14 +73,14 @@ select (t.country).city.*,(t.country).*,(country).city.* from t; logical_plan: |- LogicalProject { exprs: [Field(Field(t.country, 1:Int32), 0:Int32) as $expr1, Field(Field(t.country, 1:Int32), 1:Int32) as $expr2, Field(t.country, 0:Int32) as $expr3, Field(t.country, 1:Int32) as $expr4, Field(t.country, 2:Int32) as $expr5, Field(Field(t.country, 1:Int32), 0:Int32) as $expr6, Field(Field(t.country, 1:Int32), 1:Int32) as $expr7] } - └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } + └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } - sql: | create materialized view t as select * from s; -- select (t).country1.city.*,(t.country2).*,(country3).city.* from t; select (t.country1).city.*,(t.country2).*,(country3).city.* from t; logical_plan: |- LogicalProject { exprs: [Field(Field(t.country1, 1:Int32), 0:Int32) as $expr1, Field(Field(t.country1, 1:Int32), 1:Int32) as $expr2, Field(t.country2, 0:Int32) as $expr3, Field(t.country2, 1:Int32) as $expr4, Field(t.country2, 2:Int32) as $expr5, Field(Field(t.country3, 1:Int32), 0:Int32) as $expr6, Field(Field(t.country3, 1:Int32), 1:Int32) as $expr7] } - └─LogicalScan { table: t, columns: [t.id, t.country1, t.country2, t.country3, t.zipcode, t.rate, t._row_id] } + └─LogicalScan { table: t, columns: [t.id, t.country1, t.country2, t.country3, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } create_source: format: plain encode: protobuf @@ -112,7 +112,7 @@ logical_plan: |- LogicalProject { exprs: [Field($expr1, 1:Int32) as $expr2] } └─LogicalProject { exprs: [Field(t.country, 1:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } + └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } create_source: format: plain encode: protobuf @@ -144,7 +144,7 @@ └─LogicalProject { exprs: [min($expr1)] } └─LogicalAgg { aggs: [min($expr1)] } └─LogicalProject { exprs: [Field(t.country, 1:Int32) as $expr1] } - └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } + └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } create_source: format: plain encode: protobuf @@ -174,8 +174,8 @@ LogicalProject { exprs: [$expr1, t.id, t.country, t.zipcode, t.rate] } └─LogicalJoin { type: Inner, on: (Field($expr1, 1:Int32) = Field(t.country, 2:Int32)), output: all } ├─LogicalProject { exprs: [Field(t.country, 1:Int32) as $expr1] } - │ └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } - └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } + │ └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } create_source: format: plain encode: protobuf @@ -205,7 +205,7 @@ LogicalProject { exprs: [(min($expr1) + (max($expr1) * count(t.zipcode))) as $expr2] } └─LogicalAgg { aggs: [min($expr1), max($expr1), count(t.zipcode)] } └─LogicalProject { exprs: [Field(Field(t.country, 1:Int32), 0:Int32) as $expr1, t.zipcode] } - └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } + └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } create_source: format: plain encode: protobuf @@ -236,7 +236,7 @@ └─LogicalAgg { aggs: [count(1:Int32), count($expr1)] } └─LogicalProject { exprs: [1:Int32, Field(Field(t.country, 1:Int32), 1:Int32) as $expr1] } └─LogicalFilter { predicate: (Field(Field(t.country, 1:Int32), 0:Int32) > 1:Int32) } - └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id] } + └─LogicalScan { table: t, columns: [t.id, t.country, t.zipcode, t.rate, t._row_id, t._rw_timestamp] } create_source: format: plain encode: protobuf @@ -395,7 +395,7 @@ logical_plan: |- LogicalProject { exprs: [s.v1, s.v2, s.v3] } └─LogicalFilter { predicate: (s.v3 = Row(1:Int32, 2:Int32, Row(1:Int32, 2:Int32, 3:Int32))) } - └─LogicalScan { table: s, columns: [s.v1, s.v2, s.v3, s._row_id] } + └─LogicalScan { table: s, columns: [s.v1, s.v2, s.v3, s._row_id, s._rw_timestamp] } create_table_with_connector: format: plain encode: protobuf diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 8b9616edeae9f..7d1cf872cbed0 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -6,7 +6,7 @@ LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v2 > 1:Int32::Float64) } └─LogicalProject { exprs: [t.v1, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: merge and then eliminate sql: | create table t (v1 bigint, v2 double precision); @@ -14,7 +14,7 @@ logical_plan: |- LogicalProject { exprs: [t.v1, t.v2] } └─LogicalProject { exprs: [t.v1, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: 'LogicalScan { table: t, columns: [t.v1, t.v2] }' - sql: | create table t (v1 bigint, v2 double precision); @@ -31,7 +31,7 @@ LogicalProject { exprs: [t.v1] } └─LogicalFilter { predicate: (t.v2 > 1:Int32::Float64) } └─LogicalProject { exprs: [t.v2, t.v1] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: consecutive projects are merged sql: | create table t (v1 bigint, v2 double precision); @@ -39,7 +39,7 @@ logical_plan: |- LogicalProject { exprs: [t.v1, 2:Int32] } └─LogicalProject { exprs: [t.v1, t.v2, 1:Int32] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.v1, 2:Int32] } └─LogicalScan { table: t, columns: [t.v1] } @@ -49,7 +49,7 @@ logical_plan: |- LogicalProject { exprs: [t.v1, t.v2] } └─LogicalProject { exprs: [t.v1, t.v2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: 'LogicalScan { table: t, columns: [t.v1, t.v2] }' - name: joins sql: | @@ -59,8 +59,8 @@ LogicalProject { exprs: [t.v1, t.v2, t.v1, t.v2] } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalProject { exprs: [t.v1, t.v2] } - │ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + │ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: table alias sql: | create table t (v1 bigint, v2 double precision); @@ -69,8 +69,8 @@ LogicalProject { exprs: [t.v1, t.v2, t.v1, t.v2] } └─LogicalJoin { type: Inner, on: (t.v1 = t.v1), output: all } ├─LogicalProject { exprs: [t.v1, t.v2] } - │ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + │ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: alias less columns than available sql: | create table t (v1 bigint, v2 double precision); @@ -79,8 +79,8 @@ LogicalProject { exprs: [t.v1, t.v2, t.v1, t.v2] } └─LogicalJoin { type: Inner, on: (t.v1 = t.v1), output: all } ├─LogicalProject { exprs: [t.v1, t.v2] } - │ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + │ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } - name: alias more columns than available sql: | create table t (v1 bigint, v2 double precision); @@ -168,8 +168,8 @@ ├─LogicalProject { exprs: [sum(distinct 1:Int32)] } │ └─LogicalAgg { aggs: [sum(distinct 1:Int32)] } │ └─LogicalProject { exprs: [1:Int32] } - │ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + │ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count(1:Int32)] } └─LogicalProject { exprs: [1:Int32] } @@ -294,7 +294,7 @@ └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } ├─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all } │ └─LogicalFilter { predicate: IsNotNull(auction.date_time) } - │ └─LogicalScan { table: auction, columns: [auction.date_time, auction._row_id] } + │ └─LogicalScan { table: auction, columns: [auction.date_time, auction._row_id, auction._rw_timestamp] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr2] } └─LogicalAgg { group_key: [$expr1], aggs: [] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr1] } @@ -346,7 +346,7 @@ logical_plan: |- LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: t, columns: [t.v, t._row_id] } + ├─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } > 1:Int32) } └─LogicalAgg { aggs: [] } @@ -365,14 +365,14 @@ logical_plan: |- LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: a, columns: [a.a1, a.a2, a._row_id] } + ├─LogicalScan { table: a, columns: [a.a1, a.a2, a._row_id, a._rw_timestamp] } └─LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 2 } ├─LogicalAgg { aggs: [] } │ └─LogicalProject { exprs: [] } - │ └─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id] } + │ └─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id, b._rw_timestamp] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr1] } - └─LogicalScan { table: c, columns: [c.c1, c.c2, c._row_id] } + └─LogicalScan { table: c, columns: [c.c1, c.c2, c._row_id, c._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(a.a1, a.a1), output: [] } @@ -395,19 +395,19 @@ LogicalProject { exprs: [a.a1, a.a2] } └─LogicalFilter { predicate: (a.a1 = min(b.b1)) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: a, columns: [a.a1, a.a2, a._row_id] } + ├─LogicalScan { table: a, columns: [a.a1, a.a2, a._row_id, a._rw_timestamp] } └─LogicalProject { exprs: [min(b.b1)] } └─LogicalAgg { aggs: [min(b.b1)] } └─LogicalProject { exprs: [b.b1] } └─LogicalFilter { predicate: (b.b2 = min(b.b1)) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } - ├─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id] } + ├─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id, b._rw_timestamp] } └─LogicalProject { exprs: [min(b.b1)] } └─LogicalAgg { aggs: [min(b.b1)] } └─LogicalProject { exprs: [b.b1] } └─LogicalProject { exprs: [b.b1] } └─LogicalFilter { predicate: (b.b1 = CorrelatedInputRef { index: 0, correlated_id: 1 }) } - └─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id] } + └─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id, b._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a1, a.a1) AND (a.a1 = min(b.b1)), output: [a.a1, a.a2] } ├─LogicalScan { table: a, columns: [a.a1, a.a2] } @@ -457,12 +457,12 @@ logical_plan: |- LogicalProject { exprs: [t1.a, count] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [count] } └─LogicalAgg { aggs: [count] } └─LogicalProject { exprs: [] } └─LogicalFilter { predicate: (t1.a <> CorrelatedInputRef { index: 1, correlated_id: 1 }) } - └─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id, t1._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.b, t1.b), output: [t1.a, count(1:Int32)] } ├─LogicalScan { table: t1, columns: [t1.a, t1.b] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml index 9dc9ead95f17f..a43a1d2df7166 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml @@ -17,9 +17,9 @@ logical_plan: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalJoin { type: LeftOuter, on: true, output: all } @@ -33,10 +33,10 @@ logical_plan: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalLimit { limit: 1, offset: 0 } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalJoin { type: LeftOuter, on: true, output: all } @@ -49,10 +49,10 @@ logical_plan: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalTopN { order: [t.x ASC], limit: 1, offset: 0 } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalJoin { type: LeftOuter, on: true, output: all } @@ -65,10 +65,10 @@ logical_plan: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalTopN { order: [t.x ASC], limit: 1, offset: 0, with_ties: true } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [t.x, 1:Int32] } └─LogicalJoin { type: LeftOuter, on: true, output: all } @@ -83,9 +83,9 @@ logical_plan: |- LogicalProject { exprs: [(t.x + 1:Int32) as $expr1] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(t.x + 1:Int32) as $expr1] } └─LogicalJoin { type: LeftOuter, on: true, output: all } @@ -102,7 +102,7 @@ ├─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } │ ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } │ └─LogicalProject { exprs: [t.x] } - │ └─LogicalScan { table: t, columns: [t.x, t._row_id] } + │ └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } optimized_logical_plan_for_batch: |- @@ -119,12 +119,12 @@ logical_plan: |- LogicalProject { exprs: [(t.x + $expr1) as $expr2] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [(t.x + t.x) as $expr1] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(t.x + $expr1) as $expr2] } └─LogicalJoin { type: LeftOuter, on: true, output: all } @@ -161,7 +161,7 @@ └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftSemi, on: true, output: all } @@ -175,7 +175,7 @@ └─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [1:Int32] } └─LogicalJoin { type: LeftAnti, on: true, output: all } @@ -188,18 +188,18 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.x] } - └─LogicalScan { table: t2, columns: [t2.x, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2._row_id, t2._rw_timestamp] } - sql: | create table t(x int); select x from t where exists (select * from t); logical_plan: |- LogicalProject { exprs: [t.x] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: t, columns: [t.x, t._row_id] } + ├─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [t.x] } - └─LogicalScan { table: t, columns: [t.x, t._row_id] } + └─LogicalScan { table: t, columns: [t.x, t._row_id, t._rw_timestamp] } - sql: | create table t1(x int); create table t2(x int); @@ -208,9 +208,9 @@ LogicalProject { exprs: [t1.x] } └─LogicalFilter { predicate: (t1.x > t2.x) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.x, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.x] } - └─LogicalScan { table: t2, columns: [t2.x, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2._row_id, t2._rw_timestamp] } - sql: | select 1 where 1>0 and exists (values (1)) logical_plan: |- @@ -258,9 +258,9 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: (t1.y = t2.y), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.y] } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -268,9 +268,9 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftAnti, on: (t1.y = t2.y), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.y] } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } - sql: | create table t1 (a int); create table t2 (b int); @@ -284,13 +284,13 @@ logical_plan: |- LogicalProject { exprs: [t1.a] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.a, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.a, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [1:Int32] } └─LogicalTopN { order: [$expr2 DESC], limit: 90, offset: 0 } └─LogicalProject { exprs: [1:Int32, CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr2] } └─LogicalAgg { group_key: [$expr1], aggs: [] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 1 } as $expr1] } - └─LogicalScan { table: t2, columns: [t2.b, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.b, t2._row_id, t2._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t1.a, output: all } @@ -315,7 +315,7 @@ logical_plan: |- LogicalProject { exprs: [b.b1, ] } └─LogicalApply { type: Inner, on: true, correlated_id: 1 } - ├─LogicalScan { table: b, columns: [b.b1, b._row_id] } + ├─LogicalScan { table: b, columns: [b.b1, b._row_id, b._rw_timestamp] } └─LogicalProject { exprs: [] } └─LogicalValues { rows: [[Repeat(CorrelatedInputRef { index: 0, correlated_id: 1 }, 2:Int32)]], schema: Schema { fields: [:Varchar] } } batch_plan: |- @@ -340,16 +340,16 @@ LogicalProject { exprs: [1:Int32, 1:Int32] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } ├─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - │ ├─LogicalScan { table: dl, columns: [dl.c1, dl.c2, dl._row_id] } + │ ├─LogicalScan { table: dl, columns: [dl.c1, dl.c2, dl._row_id, dl._rw_timestamp] } │ └─LogicalLimit { limit: 1, offset: 0 } │ └─LogicalProject { exprs: [1:Int32] } │ └─LogicalFilter { predicate: (t.a = CorrelatedInputRef { index: 0, correlated_id: 1 }) } │ └─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: t, columns: [t.a, t._row_id] } - │ └─LogicalScan { table: di, columns: [di.d1, di.d2, di._row_id] } + │ ├─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } + │ └─LogicalScan { table: di, columns: [di.d1, di.d2, di._row_id, di._rw_timestamp] } └─LogicalLimit { limit: 1, offset: 0 } └─LogicalProject { exprs: [1:Int32] } └─LogicalFilter { predicate: (t.a = CorrelatedInputRef { index: 1, correlated_id: 2 }) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t, columns: [t.a, t._row_id] } - └─LogicalScan { table: di, columns: [di.d1, di.d2, di._row_id] } + ├─LogicalScan { table: t, columns: [t.a, t._row_id, t._rw_timestamp] } + └─LogicalScan { table: di, columns: [di.d1, di.d2, di._row_id, di._rw_timestamp] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 13d363a6e4877..9150ffcfd16f6 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -7,12 +7,12 @@ LogicalProject { exprs: [t1.x, t1.y] } └─LogicalFilter { predicate: (t1.x::Decimal > $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [(1.5:Decimal * min(t2.x)::Decimal) as $expr1] } └─LogicalAgg { aggs: [min(t2.x)] } └─LogicalProject { exprs: [t2.x] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) AND (t2.y = 1000:Int32) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr1] } @@ -28,13 +28,13 @@ LogicalProject { exprs: [t1.x, t1.y] } └─LogicalFilter { predicate: (t1.x > min(t2.x)) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [min(t2.x)] } └─LogicalAgg { aggs: [min(t2.x)] } └─LogicalProject { exprs: [t2.x] } └─LogicalFilter { predicate: (t2.y = $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } - ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 1, correlated_id: 1 } as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | @@ -46,19 +46,19 @@ LogicalProject { exprs: [t1.x, t1.y] } └─LogicalFilter { predicate: (t1.x > min(t2.x)) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [min(t2.x)] } └─LogicalAgg { aggs: [min(t2.x)] } └─LogicalProject { exprs: [t2.x] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) AND (CorrelatedInputRef { index: 0, correlated_id: 1 } = max(t3.x)) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } - ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } └─LogicalProject { exprs: [max(t3.x)] } └─LogicalAgg { aggs: [max(t3.x)] } └─LogicalProject { exprs: [t3.x] } └─LogicalFilter { predicate: (t3.y = CorrelatedInputRef { index: 1, correlated_id: 1 }) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } + ├─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id, t3._rw_timestamp] } └─LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | @@ -68,10 +68,10 @@ logical_plan: |- LogicalProject { exprs: [t1.x, t1.y] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.x, t2.y] } └─LogicalFilter { predicate: (t2.y = 100:Int32) AND (CorrelatedInputRef { index: 0, correlated_id: 1 } = t2.x) AND (t2.x = 1000:Int32) AND (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (t1.x = t2.x) AND (t1.y = t2.y), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -84,12 +84,12 @@ LogicalProject { exprs: [t1.x, t1.y] } └─LogicalFilter { predicate: (t1.x::Decimal > $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [(1.5:Decimal * min(t2.x)::Decimal) as $expr1] } └─LogicalAgg { aggs: [min(t2.x)] } └─LogicalProject { exprs: [t2.x] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr1] } @@ -105,12 +105,12 @@ LogicalProject { exprs: [t1.x, t1.y] } └─LogicalFilter { predicate: (t1.x > count) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [count] } └─LogicalAgg { aggs: [count] } └─LogicalProject { exprs: [] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > count(1:Int32)), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } @@ -129,12 +129,12 @@ LogicalProject { exprs: [t1.x, t1.y] } └─LogicalFilter { predicate: (t1.x > $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [(count + count) as $expr1] } └─LogicalAgg { aggs: [count] } └─LogicalProject { exprs: [] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } @@ -153,12 +153,12 @@ logical_plan: |- LogicalProject { exprs: [t1.a, array_agg(t1.a) filter(IsDistinctFrom(t1.a, 1:Int32))] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [array_agg(t1.a) filter(IsDistinctFrom(t1.a, 1:Int32))] } └─LogicalAgg { aggs: [array_agg(t1.a) filter(IsDistinctFrom(t1.a, 1:Int32))] } └─LogicalProject { exprs: [t1.a] } └─LogicalFilter { predicate: (t1.a <> CorrelatedInputRef { index: 1, correlated_id: 1 }) } - └─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.a, t1.b, t1._row_id, t1._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.b, t1.b), output: [t1.a, array_agg(t1.a) filter(IsDistinctFrom(t1.a, 1:Int32) AND IsNotNull(1:Int32))] } ├─LogicalScan { table: t1, columns: [t1.a, t1.b] } @@ -178,10 +178,10 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: (t1.y = t2.y), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.y] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } = t2.x) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x = t2.x), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -198,10 +198,10 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: (t1.y = t2.y), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.y] } └─LogicalFilter { predicate: ((CorrelatedInputRef { index: 0, correlated_id: 1 } + t2.x) = 100:Int32) AND (CorrelatedInputRef { index: 1, correlated_id: 1 } = 1000:Int32) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND ((t1.x + t2.x) = 100:Int32), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y], predicate: (t1.y = 1000:Int32) } @@ -214,10 +214,10 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: (t1.y = t2.y), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.y] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } > (t2.x + 1000:Int32)) } - └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x > $expr1), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -241,15 +241,15 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: (t1.y = t2.x), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.x] } └─LogicalFilter { predicate: (t2.y = CorrelatedInputRef { index: 1, correlated_id: 1 }) AND (t2.x > min(t3.x)) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } - ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } └─LogicalProject { exprs: [min(t3.x)] } └─LogicalAgg { aggs: [min(t3.x)] } └─LogicalProject { exprs: [t3.x] } - └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } + └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id, t3._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (t1.y = t2.x) AND (t2.y = t1.y), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -266,13 +266,13 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: (t1.y = t2.x), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.x] } └─LogicalApply { type: LeftSemi, on: (t2.y = t3.y), correlated_id: 2 } - ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } └─LogicalProject { exprs: [t3.y] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t3.y) } - └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } + └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id, t3._rw_timestamp] } - name: uncorrelated outer subquery with a correlated inner subquery sql: | create table t1(x int, y int); @@ -282,13 +282,13 @@ logical_plan: |- LogicalProject { exprs: [t1.x] } └─LogicalApply { type: LeftSemi, on: (t1.y = t2.x), correlated_id: 1 } - ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } + ├─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id, t1._rw_timestamp] } └─LogicalProject { exprs: [t2.x] } └─LogicalApply { type: LeftSemi, on: (t2.y = t3.y), correlated_id: 2 } - ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id, t2._rw_timestamp] } └─LogicalProject { exprs: [t3.y] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 2 } = t3.y) } - └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id] } + └─LogicalScan { table: t3, columns: [t3.x, t3.y, t3._row_id, t3._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftSemi, on: (t1.y = t2.x), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } @@ -364,12 +364,12 @@ └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } ├─LogicalAgg { aggs: [] } │ └─LogicalProject { exprs: [] } - │ └─LogicalScan { table: a, columns: [a.a1, a.a2, a._row_id] } + │ └─LogicalScan { table: a, columns: [a.a1, a.a2, a._row_id, a._rw_timestamp] } └─LogicalProject { exprs: [1:Int32] } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 2 } - ├─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id] } + ├─LogicalScan { table: b, columns: [b.b1, b.b2, b._row_id, b._rw_timestamp] } └─LogicalProject { exprs: [CorrelatedInputRef { index: 0, correlated_id: 2 } as $expr1] } - └─LogicalScan { table: c, columns: [c.c1, c.c2, c._row_id] } + └─LogicalScan { table: c, columns: [c.c1, c.c2, c._row_id, c._rw_timestamp] } - name: correlated column with depth=2 in HAVING sql: | create table a (a1 int, a2 int); @@ -474,13 +474,13 @@ └─LogicalFilter { predicate: (a.a3 = b.b2) AND (3:Int32 = count) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: a, columns: [a.a1, a.a2, a.a3, a._row_id] } - │ └─LogicalScan { table: b, columns: [b.b1, b.b2, b.b3, b._row_id] } + │ ├─LogicalScan { table: a, columns: [a.a1, a.a2, a.a3, a._row_id, a._rw_timestamp] } + │ └─LogicalScan { table: b, columns: [b.b1, b.b2, b.b3, b._row_id, b._rw_timestamp] } └─LogicalProject { exprs: [count] } └─LogicalAgg { aggs: [count] } └─LogicalProject { exprs: [] } - └─LogicalFilter { predicate: (CorrelatedInputRef { index: 5, correlated_id: 1 } = c.c2) AND (CorrelatedInputRef { index: 2, correlated_id: 1 } = c.c3) } - └─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id] } + └─LogicalFilter { predicate: (CorrelatedInputRef { index: 6, correlated_id: 1 } = c.c2) AND (CorrelatedInputRef { index: 2, correlated_id: 1 } = c.c3) } + └─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id, c._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, internal_last_seen_value(a.a3)) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } 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 514a56f7dff6d..37f9c7d44e3eb 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -61,21 +61,21 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:15:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } └── StreamNow { output: [now] } { tables: [ Now: 3 ] } - 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 } + Table 0 { columns: [ t1_ts, t1__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 3 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ now, _rw_timestamp ], 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 } + Table 4294967294 { columns: [ ts, t1._row_id, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - name: Temporal filter with equal condition sql: |- @@ -114,25 +114,30 @@ StreamProject { exprs: [DateTrunc('week':Varchar, now, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 5 ] } - Table 0 { columns: [ t1_ts, $expr1, t1__row_id ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 0 { columns: [ t1_ts, $expr1, t1__row_id, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr1, t1__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr1, t1__row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr2 ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr2, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr2, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ now, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ ts, t1._row_id, $expr1 ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ ts, t1._row_id, $expr1, _rw_timestamp ] + ├── primary key: [ $1 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 1, 2 ] + └── read pk prefix len hint: 2 - name: Temporal filter with `now()` in upper bound on append only table sql: |- @@ -160,21 +165,21 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '00:15:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } └── StreamNow { output: [now] } { tables: [ Now: 3 ] } - 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 } + Table 0 { columns: [ t1_ts, t1__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 3 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ now, _rw_timestamp ], 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 } + Table 4294967294 { columns: [ ts, t1._row_id, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - name: Temporal filter reorders now expressions correctly sql: | @@ -213,27 +218,27 @@ StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { tables: [ Now: 6 ] } - 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 } + Table 0 { columns: [ t1_ts, t1__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ $expr2, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 { 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 } + Table 2 { columns: [ t1_ts, t1__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 4 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 5 { columns: [ now, _rw_timestamp ], 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 6 { columns: [ now, _rw_timestamp ], 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 } + Table 4294967294 { columns: [ ts, t1._row_id, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - name: Temporal filter in on clause for inner join's left side sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/time_window.yaml b/src/frontend/planner_test/tests/testdata/output/time_window.yaml index 99131a90c6dea..25fb677a71c71 100644 --- a/src/frontend/planner_test/tests/testdata/output/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/output/time_window.yaml @@ -4,8 +4,8 @@ select * from tumble(t1, created_at, interval '3' day); logical_plan: |- LogicalProject { exprs: [t1.id, t1.created_at, $expr1, $expr2] } - └─LogicalProject { exprs: [t1.id, t1.created_at, t1._row_id, TumbleStart(t1.created_at, '3 days':Interval) as $expr1, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr2] } - └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } + └─LogicalProject { exprs: [t1.id, t1.created_at, t1._row_id, t1._rw_timestamp, TumbleStart(t1.created_at, '3 days':Interval) as $expr1, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr2] } + └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t1.id, t1.created_at, $expr1, ($expr1 + '3 days':Interval) as $expr2] } @@ -65,7 +65,7 @@ LogicalProject { exprs: [t1.id, t1.created_at, window_start, window_end] } └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id, t1._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [id, created_at, window_start, window_end, t1._row_id(hidden)], stream_key: [t1._row_id, window_start, window_end], pk_columns: [t1._row_id, window_start, window_end], pk_conflict: NoCheck } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_start, window_end, t1._row_id] } @@ -78,7 +78,7 @@ LogicalProject { exprs: [t1.id, t1.created_at, window_start] } └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id, t1._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [id, created_at, window_start, t1._row_id(hidden)], stream_key: [t1._row_id, window_start], pk_columns: [t1._row_id, window_start], pk_conflict: NoCheck } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_start, t1._row_id] } @@ -91,7 +91,7 @@ LogicalProject { exprs: [t1.id, t1.created_at, window_end] } └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id, t1._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [id, created_at, window_end, t1._row_id(hidden)], stream_key: [t1._row_id, window_end], pk_columns: [t1._row_id, window_end], pk_conflict: NoCheck } └─StreamHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at, window_end, t1._row_id] } @@ -104,7 +104,7 @@ LogicalProject { exprs: [t1.id, t1.created_at] } └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at] } └─BatchExchange { order: [], dist: Single } @@ -122,7 +122,7 @@ LogicalProject { exprs: [t1.id, t1.created_at] } └─LogicalHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─LogicalFilter { predicate: IsNotNull(t1.created_at) } - └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: [t1.id, t1.created_at] } └─BatchExchange { order: [], dist: Single } @@ -142,7 +142,7 @@ └─LogicalProject { exprs: [t.v1, window_end, t.v3] } └─LogicalHopWindow { time_col: t.v2, slide: 00:01:00, size: 00:10:00, output: all } └─LogicalFilter { predicate: IsNotNull(t.v2) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr1] } @@ -169,7 +169,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t1.id, t1.v1, t1.created_at] } └─LogicalFilter { predicate: (t1.v1 >= 10:Int32) } - └─LogicalScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t1.id, t1.v1, t1.created_at, $expr1, ($expr1 + '3 days':Interval) as $expr2] } @@ -193,7 +193,7 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [t1.id, t1.v1, t1.created_at] } └─LogicalFilter { predicate: (t1.v1 >= 10:Int32) } - └─LogicalScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id, t1._rw_timestamp] } batch_plan: |- BatchHopWindow { time_col: t1.created_at, slide: 1 day, size: 3 days, output: all } └─BatchExchange { order: [], dist: Single } @@ -223,10 +223,10 @@ logical_plan: |- LogicalProject { exprs: [t2.d1, t2.d2, t2.d3, t1.c1, t1.c2, t1.c3, window_start, window_end] } └─LogicalJoin { type: Inner, on: (t2.d1 = t1.c2), output: all } - ├─LogicalScan { table: t2, columns: [t2.d1, t2.d2, t2.d3, t2._row_id] } + ├─LogicalScan { table: t2, columns: [t2.d1, t2.d2, t2.d3, t2._row_id, t2._rw_timestamp] } └─LogicalHopWindow { time_col: t1.c3, slide: 00:10:00, size: 00:20:00, output: all } └─LogicalFilter { predicate: IsNotNull(t1.c3) } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalHopWindow { time_col: t1.c3, slide: 00:10:00, size: 00:20:00, output: all } └─LogicalFilter { predicate: IsNotNull(t1.c3) } @@ -250,10 +250,10 @@ └─LogicalJoin { type: Inner, on: (t2.d1 = t1.c2), output: all } ├─LogicalHopWindow { time_col: t2.d3, slide: 00:10:00, size: 00:20:00, output: all } │ └─LogicalFilter { predicate: IsNotNull(t2.d3) } - │ └─LogicalScan { table: t2, columns: [t2.d1, t2.d2, t2.d3, t2._row_id] } + │ └─LogicalScan { table: t2, columns: [t2.d1, t2.d2, t2.d3, t2._row_id, t2._rw_timestamp] } └─LogicalHopWindow { time_col: t1.c3, slide: 00:10:00, size: 00:20:00, output: all } └─LogicalFilter { predicate: IsNotNull(t1.c3) } - └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id, t1._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalHopWindow { time_col: t1.c3, slide: 00:10:00, size: 00:20:00, output: [t2.d1, t2.d2, t2.d3, window_start, window_end, t1.c1, t1.c2, t1.c3, window_start, window_end] } └─LogicalFilter { predicate: IsNotNull(t1.c3) } @@ -285,10 +285,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalHopWindow { time_col: alltypes2.c11, slide: 01:00:00, size: 40:00:00, output: all } │ └─LogicalFilter { predicate: IsNotNull(alltypes2.c11) } - │ └─LogicalScan { table: alltypes2, columns: [alltypes2.c1, alltypes2.c2, alltypes2.c3, alltypes2.c4, alltypes2.c5, alltypes2.c6, alltypes2.c7, alltypes2.c8, alltypes2.c9, alltypes2.c10, alltypes2.c11, alltypes2.c13, alltypes2.c14, alltypes2.c15, alltypes2.c16, alltypes2._row_id] } + │ └─LogicalScan { table: alltypes2, columns: [alltypes2.c1, alltypes2.c2, alltypes2.c3, alltypes2.c4, alltypes2.c5, alltypes2.c6, alltypes2.c7, alltypes2.c8, alltypes2.c9, alltypes2.c10, alltypes2.c11, alltypes2.c13, alltypes2.c14, alltypes2.c15, alltypes2.c16, alltypes2._row_id, alltypes2._rw_timestamp] } └─LogicalJoin { type: Inner, on: (nation.n_name = auction.item_name), output: all } - ├─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, auction.extra] } + ├─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, auction.extra, auction._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(Case((alltypes2.c2 = 32:Int32), '2022-07-29 15:06:36':Timestamp, alltypes2.c1, auction.expires, false:Boolean, (auction.expires + '-00:00:18':Interval), auction.expires) + '00:01:27':Interval) as $expr1, (618:Int32 % 311:Int32) as $expr2, 35:Int16, (Coalesce(null:Int32, null:Int32, null:Int32, null:Int32, null:Int32, 0:Int32, null:Int32, null:Int32, null:Int32, null:Int32)::Decimal - alltypes2.c7) as $expr3] } └─LogicalAgg { group_key: [auction.description, alltypes2.c7, auction.category, auction.item_name, alltypes2.c15, alltypes2.c2, alltypes2.c1, alltypes2.c9, auction.expires, alltypes2.c6], aggs: [] } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 3c43faa8d2494..e90bf2b747ccc 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -125,7 +125,7 @@ └─LogicalAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [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] } └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) as $expr1, ((lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) * (1:Int32::Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount] } └─LogicalFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal) as $expr3, (sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)::Decimal) as $expr4, (sum(lineitem.l_discount) / count(lineitem.l_discount)::Decimal) as $expr5, count] } @@ -180,15 +180,15 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count, $expr6 ], primary key: [ $0 ASC, $1 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count, $expr6, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count, $expr6 ], primary key: [ $10 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0, 1 ], read pk prefix len hint: 1, vnode column idx: 10 } + Table 1 { columns: [ lineitem_l_returnflag, lineitem_l_linestatus, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count, $expr6, _rw_timestamp ], primary key: [ $10 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0, 1 ], read pk prefix len hint: 1, vnode column idx: 10 } - Table 2 { 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 2 { 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, _rw_timestamp ], 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 3 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ 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: [], read pk prefix len hint: 2 } + 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, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [], read pk prefix len hint: 2 } - id: tpch_q2 before: @@ -247,11 +247,11 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } - │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } - │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - │ └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ │ │ ├─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment, partsupp._rw_timestamp] } + │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + │ └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } └─LogicalProject { exprs: [min(partsupp.ps_supplycost)] } └─LogicalAgg { aggs: [min(partsupp.ps_supplycost)] } └─LogicalProject { exprs: [partsupp.ps_supplycost] } @@ -259,10 +259,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } - │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment, partsupp._rw_timestamp] } + │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment] } @@ -463,97 +463,97 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, min(partsupp_ps_supplycost), partsupp_ps_partkey, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_partkey_0, _vnode ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 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: 0 } + Table 0 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, min(partsupp_ps_supplycost), partsupp_ps_partkey, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_partkey_0, _vnode, _rw_timestamp ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 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: 0 } - Table 1 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, min(partsupp_ps_supplycost), partsupp_ps_partkey, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_partkey_0, _vnode ], primary key: [ $14 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 9, 8 ], read pk prefix len hint: 1, vnode column idx: 14 } + Table 1 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, min(partsupp_ps_supplycost), partsupp_ps_partkey, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_partkey_0, _vnode, _rw_timestamp ], primary key: [ $14 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 9, 8 ], read pk prefix len hint: 1, vnode column idx: 14 } - Table 2 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_partkey, min(partsupp_ps_supplycost) ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } + Table 2 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_partkey, min(partsupp_ps_supplycost), _rw_timestamp ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } - Table 3 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 3 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), partsupp_ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 4 { columns: [ nation_n_name, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_supplycost, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_suppkey ], primary key: [ $6 ASC, $7 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ], distribution key: [ 6, 7 ], read pk prefix len hint: 3 } + Table 4 { columns: [ nation_n_name, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_supplycost, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_suppkey, _rw_timestamp ], primary key: [ $6 ASC, $7 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ], distribution key: [ 6, 7 ], read pk prefix len hint: 3 } - Table 5 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_partkey_0, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 5 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_partkey_0, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 6 { 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 6 { columns: [ part_p_partkey, part_p_mfgr, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ partsupp_ps_partkey, min(partsupp_ps_supplycost) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ partsupp_ps_partkey, min(partsupp_ps_supplycost), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ partsupp_ps_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ partsupp_ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], 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: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, region_r_regionkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ partsupp_ps_partkey, min(partsupp_ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ partsupp_ps_partkey, min(partsupp_ps_supplycost), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 13 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 14 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { 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 15 { columns: [ nation_n_nationkey, region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { 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 16 { columns: [ nation_n_nationkey, region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { 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 17 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 18 { 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 18 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - 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 19 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 20 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 21 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 22 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 22 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 23 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { 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 25 { columns: [ nation_n_nationkey, nation_n_regionkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 26 { 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 26 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 27 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 28 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 28 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 29 { columns: [ nation_n_name, supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, region_r_regionkey, nation_n_nationkey ], primary key: [ $1 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 29 { columns: [ nation_n_name, supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, region_r_regionkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 30 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 30 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 31 { 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 31 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 32 { 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 32 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ nation_n_nationkey, nation_n_name, region_r_regionkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ nation_n_nationkey, nation_n_name, region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { 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 34 { columns: [ nation_n_nationkey, region_r_regionkey, _degree, _rw_timestamp ], 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_name, supplier_s_address, supplier_s_nationkey, supplier_s_phone, supplier_s_acctbal, supplier_s_comment ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 35 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 36 { 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 36 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 37 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 37 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 38 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 38 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 39 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 39 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 40 { 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 40 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 41 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 41 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 42 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 42 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 43 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 43 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 44 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 44 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey#1, _rw_timestamp ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [], read pk prefix len hint: 9 } - id: tpch_q3 before: @@ -590,9 +590,9 @@ └─LogicalFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) AND (customer.c_custkey = orders.o_custkey) AND (lineitem.l_orderkey = orders.o_orderkey) AND (orders.o_orderdate < '1995-03-29':Date) AND (lineitem.l_shipdate > '1995-03-29':Date) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } + │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } └─LogicalProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } @@ -698,35 +698,35 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2 ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2 ], primary key: [ $4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 1, vnode column idx: 4 } + Table 1 { columns: [ lineitem_l_orderkey, sum($expr1), orders_o_orderdate, orders_o_shippriority, $expr2, _rw_timestamp ], primary key: [ $4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 1, vnode column idx: 4 } - Table 2 { columns: [ lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr1), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, sum($expr1), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ orders_o_orderkey, customer_c_custkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ orders_o_orderkey, customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { 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 5 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { 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 6 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ customer_c_custkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ customer_c_custkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ customer_c_custkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ orders_o_orderkey, orders_o_custkey, orders_o_orderdate, orders_o_shippriority ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 9 { columns: [ orders_o_orderkey, orders_o_custkey, orders_o_orderdate, orders_o_shippriority, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - 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 10 { columns: [ orders_o_custkey, orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, c_custkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, c_custkey, backfill_finished, row_count, _rw_timestamp ], 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, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ 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: 4 } + Table 4294967294 { columns: [ l_orderkey, revenue, o_orderdate, o_shippriority, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 4 } - id: tpch_q4 before: @@ -761,10 +761,10 @@ └─LogicalProject { exprs: [orders.o_orderpriority] } └─LogicalFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } └─LogicalApply { type: LeftSemi, on: true, correlated_id: 1 } - ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } + ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } └─LogicalProject { exprs: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalFilter { predicate: (lineitem.l_orderkey = CorrelatedInputRef { index: 0, correlated_id: 1 }) AND (lineitem.l_commitdate < lineitem.l_receiptdate) } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [orders.o_orderpriority], aggs: [count] } @@ -840,25 +840,25 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ orders_o_orderpriority, count, _vnode ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ orders_o_orderpriority, count, _vnode, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ orders_o_orderpriority, count, _vnode ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } + Table 1 { columns: [ orders_o_orderpriority, count, _vnode, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } - Table 2 { columns: [ orders_o_orderpriority, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ orders_o_orderpriority, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ orders_o_orderkey, orders_o_orderpriority ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ orders_o_orderkey, orders_o_orderpriority, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { 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 6 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ o_orderpriority, order_count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_orderpriority, order_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q5 before: @@ -900,12 +900,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ │ │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [nation.n_name], aggs: [sum($expr1)] } @@ -1063,70 +1063,70 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ nation_n_name, sum($expr1), $expr2 ], primary key: [ $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ nation_n_name, sum($expr1), $expr2, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ nation_n_name, sum($expr1), $expr2 ], primary key: [ $2 ASC, $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } + Table 1 { columns: [ nation_n_name, sum($expr1), $expr2, _rw_timestamp ], primary key: [ $2 ASC, $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } - Table 2 { columns: [ nation_n_name, sum($expr1), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ nation_n_name, sum($expr1), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ nation_n_nationkey, nation_n_name, region_r_regionkey ], primary key: [ $0 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 0 ], read pk prefix len hint: 2 } + Table 3 { columns: [ nation_n_nationkey, nation_n_name, region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 0 ], read pk prefix len hint: 2 } - Table 4 { columns: [ nation_n_nationkey, nation_n_nationkey_0, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 4 { columns: [ nation_n_nationkey, nation_n_nationkey_0, region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } Table 5 - ├── columns: [ customer_c_nationkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey ] + ├── columns: [ customer_c_nationkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, _rw_timestamp ] ├── primary key: [ $3 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC, $8 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 - Table 6 { columns: [ supplier_s_nationkey, customer_c_nationkey, orders_o_orderkey, orders_o_custkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 6 { columns: [ supplier_s_nationkey, customer_c_nationkey, orders_o_orderkey, orders_o_custkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - Table 7 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 9 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 10 { 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 10 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], 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, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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, customer_c_nationkey, orders_o_custkey, customer_c_custkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 13 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, customer_c_custkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 14 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 14 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 15 { 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, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 3 ], read pk prefix len hint: 2 } + Table 15 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, supplier_s_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 3 ], read pk prefix len hint: 2 } - Table 16 { columns: [ lineitem_l_orderkey, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, _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 16 { columns: [ lineitem_l_orderkey, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree, _rw_timestamp ], 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: [ 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 17 { columns: [ orders_o_orderkey, orders_o_custkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 18 { 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 18 { columns: [ orders_o_custkey, orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { 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 19 { columns: [ customer_c_custkey, customer_c_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 20 { columns: [ customer_c_custkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 21 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, c_custkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 22 { columns: [ vnode, c_custkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 23 { 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 23 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber, _rw_timestamp ], 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: [ 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 24 { columns: [ lineitem_l_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { 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 25 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 26 { columns: [ supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 27 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 28 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 28 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ n_name, revenue ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q6 before: @@ -1146,7 +1146,7 @@ └─LogicalAgg { aggs: [sum($expr1)] } └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1] } └─LogicalFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32::Decimal) } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum($expr1)] } └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1] } @@ -1184,11 +1184,11 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum($expr1)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum($expr1)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ revenue ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ revenue, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: tpch_q7 before: @@ -1246,12 +1246,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - │ │ └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + │ │ └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2)] } @@ -1395,65 +1395,65 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ nation_n_name, nation_n_name_0, $expr1, sum($expr2), $expr3 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ nation_n_name, nation_n_name_0, $expr1, sum($expr2), $expr3, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ nation_n_name, nation_n_name_0, $expr1, sum($expr2), $expr3 ], primary key: [ $4 ASC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 1, vnode column idx: 4 } + Table 1 { columns: [ nation_n_name, nation_n_name_0, $expr1, sum($expr2), $expr3, _rw_timestamp ], primary key: [ $4 ASC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 1, vnode column idx: 4 } - Table 2 { columns: [ nation_n_name, nation_n_name_0, $expr1, sum($expr2), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ nation_n_name, nation_n_name_0, $expr1, sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ nation_n_name, lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_nationkey, supplier_s_suppkey, lineitem_l_linenumber ], primary key: [ $1 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 3 { columns: [ nation_n_name, lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_nationkey, supplier_s_suppkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $1 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 4 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, 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 4 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ nation_n_name, orders_o_orderkey, nation_n_nationkey, customer_c_custkey ], primary key: [ $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 5 { columns: [ nation_n_name, orders_o_orderkey, nation_n_nationkey, customer_c_custkey, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 6 { columns: [ orders_o_orderkey, nation_n_nationkey, customer_c_custkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ orders_o_orderkey, nation_n_nationkey, customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { 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 7 { columns: [ nation_n_name, supplier_s_suppkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 8 { 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 8 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, 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 9 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, lineitem_l_linenumber, _rw_timestamp ], 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 10 { 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 10 { columns: [ lineitem_l_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { 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 11 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { 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 13 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 14 { 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 14 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 15 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 16 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 17 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 17 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 18 { 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: [ nation_n_name, customer_c_custkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 19 { 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: [ customer_c_custkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 20 { 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: [ orders_o_orderkey, orders_o_custkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 21 { 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: [ orders_o_custkey, orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { 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 22 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { 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 24 { columns: [ customer_c_custkey, customer_c_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 25 { 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 25 { columns: [ customer_c_nationkey, customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 26 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, c_custkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 27 { columns: [ vnode, c_custkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 28 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 28 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], 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: [ supp_nation, cust_nation, l_year, revenue ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 3 } - id: tpch_q8 before: @@ -1511,14 +1511,14 @@ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ │ │ ├─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } - │ │ │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - │ │ │ └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } - └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } + │ │ │ │ │ │ ├─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } + │ │ │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + │ │ │ └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } + │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } + └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment, region._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [$expr1 ASC], limit: 1, offset: 0 } └─LogicalProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4] } @@ -1715,90 +1715,90 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ $expr1, $expr4, $expr5 ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ $expr1, $expr4, $expr5, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ $expr1, $expr4, $expr5 ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } + Table 1 { columns: [ $expr1, $expr4, $expr5, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } - Table 2 { columns: [ $expr1, sum($expr3), sum($expr2), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr1, sum($expr3), sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ customer_c_custkey, region_r_regionkey, nation_n_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ customer_c_custkey, region_r_regionkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ customer_c_custkey, region_r_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ customer_c_custkey, region_r_regionkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 - ├── columns: [ 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 ] + ├── columns: [ 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, _rw_timestamp ] ├── primary key: [ $3 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 3 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ orders_o_custkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _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 6 { columns: [ orders_o_custkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], 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 7 { 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 7 { columns: [ nation_n_nationkey, region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { 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 8 { columns: [ nation_n_nationkey, region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { 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 9 { columns: [ customer_c_custkey, customer_c_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - 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 10 { columns: [ customer_c_nationkey, customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ region_r_regionkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ region_r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { 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 13 { columns: [ nation_n_nationkey, nation_n_regionkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 14 { 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: [ nation_n_regionkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ vnode, r_regionkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 15 { columns: [ vnode, r_regionkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 16 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 17 { columns: [ vnode, c_custkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 17 { columns: [ vnode, c_custkey, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 18 { 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, _rw_timestamp ], 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 19 { 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 19 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_linenumber, _degree, _rw_timestamp ], 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 20 { 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 20 { columns: [ orders_o_orderkey, orders_o_custkey, orders_o_orderdate, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { 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 22 { columns: [ nation_n_name, supplier_s_suppkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 23 { 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 23 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { 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 24 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, part_p_partkey, lineitem_l_linenumber, _rw_timestamp ], 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 25 { 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 25 { columns: [ lineitem_l_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { 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 26 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 27 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 28 { 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 28 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 29 { 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 29 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 30 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 31 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 32 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 32 { columns: [ part_p_partkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { 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 34 { columns: [ lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber, _rw_timestamp ], 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 35 { 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 35 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 36 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 36 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 37 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 37 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 38 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 38 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], 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_year, mkt_share ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_year, mkt_share, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q9 before: @@ -1849,12 +1849,12 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } - │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - │ │ └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } - │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ │ │ ├─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + │ │ └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment, partsupp._rw_timestamp] } + │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2)] } @@ -1999,65 +1999,65 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ nation_n_name, $expr1, sum($expr2), $expr3 ], primary key: [ $0 ASC, $1 DESC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ nation_n_name, $expr1, sum($expr2), $expr3, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ nation_n_name, $expr1, sum($expr2), $expr3 ], primary key: [ $3 ASC, $0 ASC, $1 DESC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 1, vnode column idx: 3 } + Table 1 { columns: [ nation_n_name, $expr1, sum($expr2), $expr3, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC, $1 DESC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 1, vnode column idx: 3 } - Table 2 { columns: [ nation_n_name, $expr1, sum($expr2), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 2 { columns: [ nation_n_name, $expr1, sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 3 { columns: [ part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 4 } + Table 3 { columns: [ part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 4 } - Table 4 { columns: [ part_p_partkey, partsupp_ps_suppkey, partsupp_ps_partkey, partsupp_ps_suppkey_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 3 ], read pk prefix len hint: 4 } + Table 4 { columns: [ part_p_partkey, partsupp_ps_suppkey, partsupp_ps_partkey, partsupp_ps_suppkey_0, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 3 ], read pk prefix len hint: 4 } - Table 5 { columns: [ 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_linenumber ], primary key: [ $3 ASC, $4 ASC, $3 ASC, $1 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1 ], read pk prefix len hint: 4 } + Table 5 { columns: [ 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_linenumber, _rw_timestamp ], primary key: [ $3 ASC, $4 ASC, $3 ASC, $1 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1 ], read pk prefix len hint: 4 } - Table 6 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_partkey_0, supplier_s_suppkey, nation_n_nationkey, orders_o_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 3 ], read pk prefix len hint: 4 } + Table 6 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_partkey_0, supplier_s_suppkey, nation_n_nationkey, orders_o_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 3 ], read pk prefix len hint: 4 } - Table 7 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ part_p_partkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { 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 9 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { 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 10 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], 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, ps_partkey, ps_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 13 { 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 13 { columns: [ nation_n_name, supplier_s_suppkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 14 { 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 14 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { 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, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 15 { 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, _rw_timestamp ], primary key: [ $2 ASC, $6 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 16 { columns: [ lineitem_l_suppkey, orders_o_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: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], 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 17 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { 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: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 20 { 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: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 21 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 22 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 23 { 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 23 { columns: [ orders_o_orderkey, orders_o_orderdate, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { 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 25 { columns: [ lineitem_l_orderkey, lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber, _rw_timestamp ], 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 26 { 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 26 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 27 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ nation, o_year, sum_profit ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ nation, o_year, sum_profit, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 2 } - id: tpch_q10 before: @@ -2104,10 +2104,10 @@ └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } + │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } └─LogicalProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } @@ -2223,51 +2223,56 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2 ], 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, 8 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2, _rw_timestamp ] + ├── 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, 8 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 1 - ├── columns: [ customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2 ] + ├── columns: [ customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2, _rw_timestamp ] ├── primary key: [ $8 ASC, $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, 8 ] ├── distribution key: [ 0 ] ├── read pk prefix len hint: 1 └── vnode column idx: 8 - Table 2 { columns: [ customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr1), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 7 } + Table 2 { columns: [ customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, sum($expr1), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 7 } - Table 3 { columns: [ nation_n_name, customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, nation_n_nationkey ], primary key: [ $1 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 3 { columns: [ nation_n_name, customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, nation_n_nationkey, _rw_timestamp ], primary key: [ $1 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 4 { 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 4 { columns: [ customer_c_custkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ lineitem_l_extendedprice, lineitem_l_discount, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, orders_o_orderkey ], primary key: [ $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 5 { columns: [ lineitem_l_extendedprice, lineitem_l_discount, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, orders_o_orderkey, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 6 { columns: [ orders_o_custkey, 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 6 { columns: [ orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { 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 7 { columns: [ nation_n_nationkey, nation_n_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ customer_c_custkey, customer_c_name, customer_c_address, customer_c_nationkey, customer_c_phone, customer_c_acctbal, customer_c_comment ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 9 { columns: [ customer_c_custkey, customer_c_name, customer_c_address, customer_c_nationkey, customer_c_phone, customer_c_acctbal, customer_c_comment, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - 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 10 { columns: [ customer_c_nationkey, customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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, c_custkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, c_custkey, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 13 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber, _rw_timestamp ], 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 14 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], 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 15 { columns: [ orders_o_orderkey, orders_o_custkey, _rw_timestamp ], 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 16 { columns: [ orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 17 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 18 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 18 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], 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: [ 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: 8 } + Table 4294967294 { columns: [ c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment, _rw_timestamp ], 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: 8 } - id: tpch_q11 before: @@ -2312,18 +2317,18 @@ │ └─LogicalFilter { predicate: (partsupp.ps_suppkey = supplier.s_suppkey) AND (supplier.s_nationkey = nation.n_nationkey) AND (nation.n_name = 'ARGENTINA':Varchar) } │ └─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } - │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment, partsupp._rw_timestamp] } + │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } └─LogicalProject { exprs: [(sum($expr2) * 0.0001000000:Decimal) as $expr3] } └─LogicalAgg { aggs: [sum($expr2)] } └─LogicalProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2] } └─LogicalFilter { predicate: (partsupp.ps_suppkey = supplier.s_suppkey) AND (supplier.s_nationkey = nation.n_nationkey) AND (nation.n_name = 'ARGENTINA':Varchar) } └─LogicalJoin { type: Inner, on: true, output: all } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } - │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment, partsupp._rw_timestamp] } + │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } └─LogicalJoin { type: Inner, on: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } @@ -2470,41 +2475,41 @@ └── StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } └── StreamExchange NoShuffle from 3 - Table 0 { columns: [ partsupp_ps_partkey, sum($expr1), _vnode ], primary key: [ $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ partsupp_ps_partkey, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ partsupp_ps_partkey, sum($expr1), _vnode ], primary key: [ $2 ASC, $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } + Table 1 { columns: [ partsupp_ps_partkey, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $2 ASC, $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } - Table 2 { columns: [ partsupp_ps_partkey, sum($expr1) ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ partsupp_ps_partkey, sum($expr1), _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr3 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 3 { columns: [ $expr3, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4 { columns: [ partsupp_ps_partkey, sum($expr1), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ partsupp_ps_partkey, sum($expr1), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $3 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 5 { columns: [ partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 6 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty, partsupp_ps_supplycost ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 9 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty, partsupp_ps_supplycost, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 10 { 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 10 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { 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 11 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 14 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 15 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ sum(sum($expr2)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 16 { columns: [ sum(sum($expr2)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ ps_partkey, value ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ ps_partkey, value, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q12 before: @@ -2546,8 +2551,8 @@ └─LogicalProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2] } └─LogicalFilter { predicate: (orders.o_orderkey = lineitem.l_orderkey) AND 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 < ('1994-01-01':Date + '1 year':Interval)) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2)] } @@ -2619,25 +2624,25 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ lineitem_l_shipmode, sum($expr1), sum($expr2), $expr3 ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ lineitem_l_shipmode, sum($expr1), sum($expr2), $expr3, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ lineitem_l_shipmode, sum($expr1), sum($expr2), $expr3 ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 3 } + Table 1 { columns: [ lineitem_l_shipmode, sum($expr1), sum($expr2), $expr3, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 3 } - Table 2 { columns: [ lineitem_l_shipmode, sum($expr1), sum($expr2), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ lineitem_l_shipmode, sum($expr1), sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ orders_o_orderkey, orders_o_orderpriority ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ orders_o_orderkey, orders_o_orderpriority, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ orders_o_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ lineitem_l_orderkey, lineitem_l_shipmode, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ lineitem_l_orderkey, lineitem_l_shipmode, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { 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 6 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 7 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ l_shipmode, high_line_count, low_line_count ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ l_shipmode, high_line_count, low_line_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q13 before: @@ -2673,8 +2678,8 @@ └─LogicalAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey)] } └─LogicalProject { exprs: [customer.c_custkey, orders.o_orderkey] } └─LogicalJoin { type: LeftOuter, on: (customer.c_custkey = orders.o_custkey) AND Not(Like(orders.o_comment, '%:1%:2%':Varchar)), output: all } - ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } + ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } + └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } @@ -2751,27 +2756,27 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ count(orders_o_orderkey), count, _vnode ], primary key: [ $1 DESC, $0 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ count(orders_o_orderkey), count, _vnode, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ count(orders_o_orderkey), count, _vnode ], primary key: [ $2 ASC, $1 DESC, $0 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } + Table 1 { columns: [ count(orders_o_orderkey), count, _vnode, _rw_timestamp ], primary key: [ $2 ASC, $1 DESC, $0 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } - Table 2 { columns: [ count(orders_o_orderkey), count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ count(orders_o_orderkey), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ customer_c_custkey, count(orders_o_orderkey), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ customer_c_custkey, count(orders_o_orderkey), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ customer_c_custkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ customer_c_custkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ customer_c_custkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { 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 6 { columns: [ orders_o_orderkey, orders_o_custkey, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 7 { 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 7 { columns: [ orders_o_custkey, orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ vnode, c_custkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ vnode, c_custkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], 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: [ c_count, custdist ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } - id: tpch_q14 before: @@ -2796,8 +2801,8 @@ └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)), 0:Int32::Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) as $expr2] } └─LogicalFilter { predicate: (lineitem.l_partkey = part.p_partkey) AND (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } + ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [((100.00:Decimal * sum($expr1)) / sum($expr2)) as $expr3] } └─LogicalAgg { aggs: [sum($expr1), sum($expr2)] } @@ -2860,21 +2865,21 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum($expr1)), sum(sum($expr2)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum($expr1)), sum(sum($expr2)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { 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 2 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ part_p_partkey, part_p_type ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ part_p_partkey, part_p_type, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], 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: [ promo_revenue ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ promo_revenue, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: tpch_q15 before: @@ -2918,13 +2923,13 @@ └─LogicalFilter { predicate: (supplier.s_suppkey = lineitem.l_suppkey) AND (sum($expr1) = max(sum($expr1))) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } + │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } │ └─LogicalShare { id: 5 } │ └─LogicalProject { exprs: [lineitem.l_suppkey, sum($expr1)] } │ └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1)] } │ └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) as $expr1] } │ └─LogicalFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } - │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } └─LogicalProject { exprs: [max(sum($expr1))] } └─LogicalAgg { aggs: [max(sum($expr1))] } └─LogicalProject { exprs: [sum($expr1)] } @@ -2933,7 +2938,7 @@ └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1)] } └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) as $expr1] } └─LogicalFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } └─LogicalJoin { type: Inner, on: (sum($expr1) = max(sum($expr2))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } @@ -3029,41 +3034,41 @@ └── StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as _vnode] } └── StreamExchange NoShuffle from 4 - Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), _vnode ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), _vnode ], primary key: [ $5 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 1, vnode column idx: 5 } + Table 1 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $5 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 1, vnode column idx: 5 } - Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), lineitem_l_suppkey ], primary key: [ $4 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), lineitem_l_suppkey, _rw_timestamp ], primary key: [ $4 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 3 { columns: [ sum($expr1), supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ sum($expr1), supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ max(max(sum($expr1))) ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ max(max(sum($expr1))), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ max(max(sum($expr1))), _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ max(max(sum($expr1))), _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ lineitem_l_suppkey, sum($expr1) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ lineitem_l_suppkey, sum($expr1), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ lineitem_l_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ lineitem_l_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ lineitem_l_suppkey, sum($expr1), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ lineitem_l_suppkey, sum($expr1), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 13 { columns: [ max(sum($expr1)), _vnode ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 13 { columns: [ max(sum($expr1)), _vnode, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 14 { columns: [ max(max(sum($expr1))), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 14 { columns: [ max(max(sum($expr1))), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 15 { columns: [ _vnode, 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 15 { columns: [ _vnode, sum($expr1), lineitem_l_suppkey, _rw_timestamp ], 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 16 { columns: [ _vnode, 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 16 { columns: [ _vnode, max(sum($expr1)), count, _rw_timestamp ], 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 ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q16 before: @@ -3108,11 +3113,11 @@ └─LogicalFilter { predicate: (part.p_partkey = partsupp.ps_partkey) AND (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) } └─LogicalApply { type: LeftAnti, on: (partsupp.ps_suppkey = supplier.s_suppkey), correlated_id: 1 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } - │ └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } + │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment, partsupp._rw_timestamp] } + │ └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } └─LogicalProject { exprs: [supplier.s_suppkey] } └─LogicalFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } - └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } + └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [count(partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(partsupp.ps_suppkey)] } @@ -3203,37 +3208,43 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ part_p_brand, part_p_type, part_p_size, count(distinct partsupp_ps_suppkey), $expr1 ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ part_p_brand, part_p_type, part_p_size, count(distinct partsupp_ps_suppkey), $expr1, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ part_p_brand, part_p_type, part_p_size, count(distinct partsupp_ps_suppkey), $expr1 ], primary key: [ $4 ASC, $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 1, vnode column idx: 4 } + Table 1 + ├── columns: [ part_p_brand, part_p_type, part_p_size, count(distinct partsupp_ps_suppkey), $expr1, _rw_timestamp ] + ├── primary key: [ $4 ASC, $3 DESC, $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 0, 1, 2 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 4 - Table 2 { columns: [ part_p_brand, part_p_type, part_p_size, count(distinct partsupp_ps_suppkey), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ part_p_brand, part_p_type, part_p_size, count(distinct partsupp_ps_suppkey), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 3 { columns: [ part_p_brand, part_p_type, part_p_size, partsupp_ps_suppkey, count_for_agg_call_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 4 } + Table 3 { columns: [ part_p_brand, part_p_type, part_p_size, partsupp_ps_suppkey, count_for_agg_call_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 4 } - Table 4 { columns: [ partsupp_ps_suppkey, part_p_brand, part_p_type, part_p_size, partsupp_ps_partkey, part_p_partkey ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ partsupp_ps_suppkey, part_p_brand, part_p_type, part_p_size, partsupp_ps_partkey, part_p_partkey, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { 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 5 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ supplier_s_suppkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ supplier_s_suppkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { 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: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ part_p_partkey, part_p_brand, part_p_type, part_p_size ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ part_p_partkey, part_p_brand, part_p_type, part_p_size, _rw_timestamp ], primary key: [ $0 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 11 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 13 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 13 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 14 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ 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: [], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ p_brand, p_type, p_size, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 4 } - id: tpch_q17 before: @@ -3263,13 +3274,13 @@ └─LogicalFilter { predicate: (part.p_partkey = lineitem.l_partkey) AND (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) AND (lineitem.l_quantity < $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - │ └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } + │ ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + │ └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } └─LogicalProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } └─LogicalAgg { aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } └─LogicalProject { exprs: [lineitem.l_quantity] } - └─LogicalFilter { predicate: (lineitem.l_partkey = CorrelatedInputRef { index: 16, correlated_id: 1 }) } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + └─LogicalFilter { predicate: (lineitem.l_partkey = CorrelatedInputRef { index: 17, correlated_id: 1 }) } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(sum(lineitem.l_extendedprice) / 7.0:Decimal) as $expr2] } └─LogicalAgg { aggs: [sum(lineitem.l_extendedprice)] } @@ -3361,33 +3372,33 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum(lineitem_l_extendedprice)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum(lineitem_l_extendedprice)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ lineitem_l_quantity, lineitem_l_extendedprice, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 1 { columns: [ lineitem_l_quantity, lineitem_l_extendedprice, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 2 { columns: [ part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr1, lineitem_l_partkey ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr1, lineitem_l_partkey, _rw_timestamp ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 4 { columns: [ lineitem_l_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ lineitem_l_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { 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 6 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ part_p_partkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 10 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], 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: [ lineitem_l_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 11 { columns: [ lineitem_l_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ avg_yearly ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + Table 4294967294 { columns: [ avg_yearly, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: tpch_q18 before: @@ -3436,14 +3447,14 @@ └─LogicalApply { type: LeftSemi, on: (orders.o_orderkey = lineitem.l_orderkey), correlated_id: 1 } ├─LogicalJoin { type: Inner, on: true, output: all } │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } - │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + │ │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } + │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } └─LogicalProject { exprs: [lineitem.l_orderkey] } └─LogicalFilter { predicate: (sum(lineitem.l_quantity) > 1:Int32::Decimal) } └─LogicalAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity)] } └─LogicalProject { exprs: [lineitem.l_orderkey, lineitem.l_quantity] } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } └─LogicalAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [sum(lineitem.l_quantity)] } @@ -3554,58 +3565,63 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1 ], primary key: [ $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1, _rw_timestamp ] + ├── primary key: [ $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 1 - ├── columns: [ customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1 ] + ├── columns: [ customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1, _rw_timestamp ] ├── primary key: [ $6 ASC, $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] ├── distribution key: [ 2 ] ├── read pk prefix len hint: 1 └── vnode column idx: 6 - Table 2 { columns: [ customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, sum(lineitem_l_quantity), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6 ], distribution key: [ 2 ], read pk prefix len hint: 5 } + Table 2 { columns: [ customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, sum(lineitem_l_quantity), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6 ], distribution key: [ 2 ], read pk prefix len hint: 5 } Table 3 - ├── columns: [ customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ] + ├── columns: [ customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber, _rw_timestamp ] ├── primary key: [ $2 ASC, $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 2 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ orders_o_orderkey, customer_c_custkey, 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 4 { columns: [ orders_o_orderkey, customer_c_custkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ lineitem_l_orderkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ lineitem_l_orderkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ lineitem_l_orderkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ lineitem_l_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 7 { columns: [ customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 8 { columns: [ orders_o_orderkey, customer_c_custkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ orders_o_orderkey, customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { 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 10 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ customer_c_custkey, customer_c_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ customer_c_custkey, customer_c_name, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ customer_c_custkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ customer_c_custkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ orders_o_orderkey, orders_o_custkey, orders_o_totalprice, orders_o_orderdate ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 13 { columns: [ orders_o_orderkey, orders_o_custkey, orders_o_totalprice, orders_o_orderdate, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - 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 14 { columns: [ orders_o_custkey, orders_o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ vnode, c_custkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 15 { columns: [ vnode, c_custkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 16 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 17 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 17 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], 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 18 { columns: [ lineitem_l_orderkey, sum(lineitem_l_quantity), count, _rw_timestamp ], 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, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ 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 } + Table 4294967294 { columns: [ c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity, _rw_timestamp ], 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 } - id: tpch_q19 before: @@ -3652,8 +3668,8 @@ └─LogicalProject { exprs: [(lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) as $expr1] } └─LogicalFilter { predicate: (part.p_partkey = lineitem.l_partkey) AND (part.p_size >= 1:Int32) AND In(lineitem.l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (lineitem.l_shipinstruct = 'DELIVER IN PERSON':Varchar) AND ((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND ((lineitem.l_quantity >= 1:Int32::Decimal) AND (lineitem.l_quantity <= 11:Int32::Decimal))) AND (part.p_size <= 5:Int32)) OR ((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND ((lineitem.l_quantity >= 30:Int32::Decimal) AND (lineitem.l_quantity <= 40:Int32::Decimal))) AND (part.p_size <= 10:Int32))) OR ((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND ((lineitem.l_quantity >= 10:Int32::Decimal) AND (lineitem.l_quantity <= 20:Int32::Decimal))) AND (part.p_size <= 15:Int32))) } └─LogicalJoin { type: Inner, on: true, output: all } - ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } + ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [sum($expr1)] } └─LogicalProject { exprs: [(lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) as $expr1] } @@ -3715,21 +3731,21 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ sum(sum($expr1)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum(sum($expr1)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { 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 2 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ part_p_partkey, part_p_brand, part_p_size, part_p_container ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ part_p_partkey, part_p_brand, part_p_size, part_p_container, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], 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 } + Table 4294967294 { columns: [ revenue, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - id: tpch_q20 before: @@ -3741,21 +3757,21 @@ └─LogicalFilter { predicate: (supplier.s_nationkey = nation.n_nationkey) AND (nation.n_name = 'KENYA':Varchar) } └─LogicalApply { type: LeftSemi, on: (supplier.s_suppkey = partsupp.ps_suppkey), correlated_id: 1 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } └─LogicalProject { exprs: [partsupp.ps_suppkey] } └─LogicalFilter { predicate: (lineitem.l_partkey = partsupp.ps_partkey) AND (lineitem.l_suppkey = partsupp.ps_suppkey) AND (partsupp.ps_availqty::Decimal > $expr1) } └─LogicalApply { type: LeftSemi, on: (partsupp.ps_partkey = part.p_partkey), correlated_id: 2 } ├─LogicalJoin { type: Inner, on: true, output: all } - │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment] } + │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_comment, partsupp._rw_timestamp] } │ └─LogicalProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr1] } │ └─LogicalAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity)] } │ └─LogicalProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } │ └─LogicalFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } - │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } └─LogicalProject { exprs: [part.p_partkey] } └─LogicalFilter { predicate: Like(part.p_name, 'forest%':Varchar) } - └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } + └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment, part._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } └─LogicalJoin { type: LeftSemi, on: (supplier.s_suppkey = partsupp.ps_suppkey), output: [supplier.s_name, supplier.s_address] } @@ -3898,55 +3914,55 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, supplier_s_nationkey, _vnode ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, supplier_s_nationkey, _vnode, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, supplier_s_nationkey, _vnode ], primary key: [ $4 ASC, $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 4 } + Table 1 { columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, supplier_s_nationkey, _vnode, _rw_timestamp ], primary key: [ $4 ASC, $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 4 } - Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { 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 5 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 6 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, _rw_timestamp ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 7 { 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 7 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ nation_n_nationkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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, lineitem_l_partkey, lineitem_l_suppkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, lineitem_l_partkey, lineitem_l_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - 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 13 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ part_p_partkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ part_p_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { 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 16 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, $expr1, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 17 { 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 17 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 18 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, $expr2 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 18 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 19 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, _degree ], 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, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 20 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 21 { columns: [ lineitem_l_partkey, lineitem_l_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 21 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, sum(lineitem_l_quantity), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 22 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 22 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 23 { columns: [ vnode, p_partkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 23 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q21 before: @@ -4003,16 +4019,16 @@ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ ├─LogicalJoin { type: Inner, on: true, output: all } │ │ │ ├─LogicalJoin { type: Inner, on: true, output: all } - │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } - │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } + │ │ │ │ ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, supplier._rw_timestamp] } + │ │ │ │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } + │ │ │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } + │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment, nation._rw_timestamp] } │ └─LogicalProject { exprs: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - │ └─LogicalFilter { predicate: (lineitem.l_orderkey = CorrelatedInputRef { index: 7, correlated_id: 1 }) AND (lineitem.l_suppkey <> CorrelatedInputRef { index: 9, correlated_id: 1 }) } - │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + │ └─LogicalFilter { predicate: (lineitem.l_orderkey = CorrelatedInputRef { index: 8, correlated_id: 1 }) AND (lineitem.l_suppkey <> CorrelatedInputRef { index: 10, correlated_id: 1 }) } + │ └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } └─LogicalProject { exprs: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } - └─LogicalFilter { predicate: (lineitem.l_orderkey = CorrelatedInputRef { index: 7, correlated_id: 2 }) AND (lineitem.l_suppkey <> CorrelatedInputRef { index: 9, correlated_id: 2 }) AND (lineitem.l_receiptdate > lineitem.l_commitdate) } - └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } + └─LogicalFilter { predicate: (lineitem.l_orderkey = CorrelatedInputRef { index: 8, correlated_id: 2 }) AND (lineitem.l_suppkey <> CorrelatedInputRef { index: 10, correlated_id: 2 }) AND (lineitem.l_receiptdate > lineitem.l_commitdate) } + └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment, lineitem._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } └─LogicalAgg { group_key: [supplier.s_name], aggs: [count] } @@ -4171,65 +4187,65 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_name, count, _vnode ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ supplier_s_name, count, _vnode, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ supplier_s_name, count, _vnode ], primary key: [ $2 ASC, $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } + Table 1 { columns: [ supplier_s_name, count, _vnode, _rw_timestamp ], primary key: [ $2 ASC, $1 DESC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 2 } - Table 2 { columns: [ supplier_s_name, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ supplier_s_name, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $1 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 4 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, 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 4 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], 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 5 { 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 5 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { 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 6 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 7 { columns: [ supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $1 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 8 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, 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 8 { columns: [ lineitem_l_orderkey, nation_n_nationkey, supplier_s_suppkey, orders_o_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], 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 9 { 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 9 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { 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 10 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ supplier_s_suppkey, supplier_s_name, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ supplier_s_suppkey, supplier_s_name, nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], 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: [ supplier_s_suppkey, nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 13 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 14 { columns: [ lineitem_l_suppkey, orders_o_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 14 { columns: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ nation_n_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], 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 16 { columns: [ nation_n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 17 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], 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 18 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ vnode, n_nationkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 19 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], 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: [ vnode, s_suppkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 20 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], 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: [ orders_o_orderkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ orders_o_orderkey, _rw_timestamp ], 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 22 { columns: [ orders_o_orderkey, _degree, _rw_timestamp ], 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 23 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, lineitem_l_linenumber, _rw_timestamp ], 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 24 { columns: [ lineitem_l_orderkey, lineitem_l_linenumber, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ vnode, o_orderkey, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 25 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 26 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 26 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 27 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 27 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ s_name, numwait ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } - id: tpch_q22 before: @@ -4282,15 +4298,15 @@ └─LogicalFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) AND (customer.c_acctbal > $expr1) } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 2, max_one_row: true } ├─LogicalApply { type: LeftAnti, on: true, correlated_id: 1 } - │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } + │ ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } │ └─LogicalProject { exprs: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } │ └─LogicalFilter { predicate: (orders.o_custkey = CorrelatedInputRef { index: 0, correlated_id: 1 }) } - │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } + │ └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment, orders._rw_timestamp] } └─LogicalProject { exprs: [(sum(customer.c_acctbal) / count(customer.c_acctbal)::Decimal) as $expr1] } └─LogicalAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } └─LogicalProject { exprs: [customer.c_acctbal] } └─LogicalFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } + └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment, customer._rw_timestamp] } optimized_logical_plan_for_batch: |- LogicalTopN { order: [$expr2 ASC], limit: 1, offset: 0 } └─LogicalAgg { group_key: [$expr2], aggs: [count, sum(customer.c_acctbal)] } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml index 7b192ccceed49..772ff95b58f9e 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -375,83 +375,83 @@ StreamFilter { predicate: IsNotNull(ps_partkey) } └── StreamExchange NoShuffle from 7 - Table 0 { columns: [ p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id ], primary key: [ $0 ASC, $3 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } + Table 0 { columns: [ p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } - Table 1 { columns: [ p_partkey, min(ps_supplycost), ps_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 1 { columns: [ p_partkey, min(ps_supplycost), ps_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 2 { columns: [ n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, s_suppkey, _row_id_2 ], primary key: [ $6 ASC, $7 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $14 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 6, 7 ], read pk prefix len hint: 3 } + Table 2 { columns: [ n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, s_suppkey, _row_id_2, _rw_timestamp ], primary key: [ $6 ASC, $7 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $14 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 6, 7 ], read pk prefix len hint: 3 } - Table 3 { columns: [ ps_partkey, ps_supplycost, ps_partkey_0, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, _row_id_2, s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } + Table 3 { columns: [ ps_partkey, ps_supplycost, ps_partkey_0, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, _row_id_2, s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 4 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ p_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ p_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ ps_partkey, min(ps_supplycost) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ ps_partkey, min(ps_supplycost), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ ps_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 9 { columns: [ ps_partkey, min(ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ ps_partkey, min(ps_supplycost), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id_0 ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 10 { columns: [ ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id_0, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 11 { columns: [ s_nationkey, _row_id, _row_id_0, ps_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 11 { columns: [ s_nationkey, _row_id, _row_id_0, ps_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ n_nationkey, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ n_nationkey, _row_id, r_regionkey, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _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: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id ], primary key: [ $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 14 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 15 { columns: [ ps_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ ps_suppkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ s_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ s_suppkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 18 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 19 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 19 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 20 { columns: [ r_regionkey, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ r_regionkey, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ r_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 22 { columns: [ n_nationkey, n_name, n_regionkey, _row_id, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 23 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ n_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 24 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 25 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 25 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 26 { columns: [ n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1 ], primary key: [ $1 ASC, $7 ASC, $8 ASC, $9 ASC, $11 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 26 { columns: [ n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1, _rw_timestamp ], primary key: [ $1 ASC, $7 ASC, $8 ASC, $9 ASC, $11 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 27 { columns: [ s_suppkey, _row_id, _row_id_0, r_regionkey, _row_id_1, n_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 27 { columns: [ s_suppkey, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, _degree, _rw_timestamp ], 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 28 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id ], primary key: [ $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 28 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id, _rw_timestamp ], primary key: [ $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 29 { columns: [ ps_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ ps_suppkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ n_nationkey, n_name, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 30 { columns: [ n_nationkey, n_name, _row_id, r_regionkey, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 31 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 31 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 32 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 32 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 33 { columns: [ s_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ s_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { columns: [ r_regionkey, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 34 { columns: [ r_regionkey, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 35 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 35 { columns: [ r_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 36 { columns: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 36 { columns: [ n_nationkey, n_name, n_regionkey, _row_id, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 37 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 37 { columns: [ n_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], read pk prefix len hint: 14 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, _rw_timestamp ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], read pk prefix len hint: 14 } - id: tpch_q5 before: @@ -630,76 +630,81 @@ StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { tables: [ Source: 26 ] } - Table 0 { columns: [ n_name, sum($expr1), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ n_name, sum($expr1), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ n_nationkey, n_name, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ n_nationkey, n_name, _row_id, r_regionkey, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 0 ], read pk prefix len hint: 2 } - Table 2 { columns: [ n_nationkey, n_nationkey_0, _row_id, _row_id_0, r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ n_nationkey, n_nationkey_0, _row_id, _row_id_0, r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } Table 3 - ├── columns: [ c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id_0, o_custkey, o_orderkey, _row_id_1, _row_id_2, l_suppkey ] + ├── columns: [ c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id_0, o_custkey, o_orderkey, _row_id_1, _row_id_2, l_suppkey, _rw_timestamp ] ├── primary key: [ $3 ASC, $0 ASC, $4 ASC, $5 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 Table 4 - ├── columns: [ s_nationkey, c_nationkey, _row_id, _row_id_0, o_custkey, _row_id_1, _row_id_2, l_suppkey, o_orderkey, _degree ] + ├── columns: [ s_nationkey, c_nationkey, _row_id, _row_id_0, o_custkey, _row_id_1, _row_id_2, l_suppkey, o_orderkey, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ] ├── value indices: [ 9 ] ├── distribution key: [ 1, 0 ] └── read pk prefix len hint: 2 - Table 5 { columns: [ r_regionkey, r_name, r_comment, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ r_regionkey, r_name, r_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ r_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 7 { columns: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id, _rw_timestamp ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 8 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ n_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 9 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 10 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 10 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 11 { columns: [ o_orderkey, c_nationkey, _row_id, o_custkey, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 11 { columns: [ o_orderkey, c_nationkey, _row_id, o_custkey, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 12 { columns: [ o_orderkey, c_nationkey, _row_id, _row_id_0, o_custkey, _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 12 { columns: [ o_orderkey, c_nationkey, _row_id, _row_id_0, o_custkey, _degree, _rw_timestamp ], 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 13 { columns: [ l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id_0 ], 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 13 { columns: [ l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id_0, _rw_timestamp ], 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: [ l_orderkey, s_nationkey, _row_id, _row_id_0, 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 14 { columns: [ l_orderkey, s_nationkey, _row_id, _row_id_0, l_suppkey, _degree, _rw_timestamp ], 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: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id ], primary key: [ $1 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 15 + ├── columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $9 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] + ├── distribution key: [ 1 ] + └── read pk prefix len hint: 1 - Table 16 { columns: [ o_custkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ o_custkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id ], primary key: [ $0 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ c_custkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ c_custkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 19 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 20 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 20 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 21 - ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id ] + ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id, _rw_timestamp ] ├── primary key: [ $2 ASC, $16 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ] ├── distribution key: [ 2 ] └── read pk prefix len hint: 1 - Table 22 { columns: [ l_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 22 { columns: [ l_suppkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ s_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ s_suppkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 25 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 26 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 26 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], 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: 2 } + Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - id: tpch_q7 before: @@ -900,74 +905,84 @@ StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } { tables: [ Source: 25 ] } - Table 0 { columns: [ n_name, n_name_0, $expr1, sum($expr2), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ n_name, n_name_0, $expr1, sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 1 - ├── columns: [ n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _row_id_0, n_nationkey, s_suppkey, _row_id_1 ] + ├── columns: [ n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _row_id_0, n_nationkey, s_suppkey, _row_id_1, _rw_timestamp ] ├── primary key: [ $1 ASC, $5 ASC, $6 ASC, $7 ASC, $9 ASC, $8 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 1 - Table 2 { columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, s_suppkey, _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 2 { columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, s_suppkey, _degree, _rw_timestamp ], 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 3 { columns: [ n_name, o_orderkey, _row_id, _row_id_0, n_nationkey, c_custkey, _row_id_1 ], primary key: [ $1 ASC, $2 ASC, $3 ASC, $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 3 + ├── columns: [ n_name, o_orderkey, _row_id, _row_id_0, n_nationkey, c_custkey, _row_id_1, _rw_timestamp ] + ├── primary key: [ $1 ASC, $2 ASC, $3 ASC, $4 ASC, $6 ASC, $5 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 1 ] + └── read pk prefix len hint: 1 - Table 4 { columns: [ o_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, c_custkey, _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 4 { columns: [ o_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, c_custkey, _degree, _rw_timestamp ], 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 5 { columns: [ n_name, s_suppkey, _row_id, n_nationkey, _row_id_0 ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 5 { columns: [ n_name, s_suppkey, _row_id, n_nationkey, _row_id_0, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 6 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 - ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id ] + ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id, _rw_timestamp ] ├── primary key: [ $2 ASC, $16 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ] ├── distribution key: [ 2 ] └── read pk prefix len hint: 1 - Table 8 { columns: [ l_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ l_suppkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ n_nationkey, n_name, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ n_nationkey, n_name, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ n_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ n_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 11 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 12 { columns: [ s_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ s_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 13 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 14 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 14 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 15 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 15 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 16 { columns: [ n_name, c_custkey, _row_id, n_nationkey, _row_id_0 ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 16 { columns: [ n_name, c_custkey, _row_id, n_nationkey, _row_id_0, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 17 { columns: [ c_custkey, _row_id, _row_id_0, n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ c_custkey, _row_id, _row_id_0, n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 18 - ├── columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id ] + ├── columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $9 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 1 - Table 19 { columns: [ o_custkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 19 { columns: [ o_custkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 20 { columns: [ n_nationkey, n_name, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ n_nationkey, n_name, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ n_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ n_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id ], primary key: [ $3 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 22 + ├── columns: [ c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id, _rw_timestamp ] + ├── primary key: [ $3 ASC, $8 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] + ├── distribution key: [ 3 ] + └── read pk prefix len hint: 1 - Table 23 { columns: [ c_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ c_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 24 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 24 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 25 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 25 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], 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 } + Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue, _rw_timestamp ], 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 } - id: tpch_q8 before: @@ -1219,99 +1234,99 @@ StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } { tables: [ Source: 35 ] } - Table 0 { columns: [ $expr1, sum($expr3), sum($expr2), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ $expr1, sum($expr3), sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ c_custkey, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ c_custkey, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ c_custkey, _row_id, _row_id_0, r_regionkey, _row_id_1, n_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 2 { columns: [ c_custkey, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, _degree, _rw_timestamp ], 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 3 - ├── columns: [ n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, p_partkey, s_suppkey, l_orderkey, _row_id_3 ] + ├── columns: [ n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, p_partkey, s_suppkey, l_orderkey, _row_id_3, _rw_timestamp ] ├── primary key: [ $3 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $12 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 3 ] └── read pk prefix len hint: 1 Table 4 - ├── columns: [ o_custkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, p_partkey, s_suppkey, _row_id_3, l_orderkey, _degree ] + ├── columns: [ o_custkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, p_partkey, s_suppkey, _row_id_3, l_orderkey, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ] ├── value indices: [ 10 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 5 { columns: [ n_nationkey, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ n_nationkey, _row_id, r_regionkey, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id ], primary key: [ $3 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 7 { columns: [ c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 8 { columns: [ c_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ c_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ r_regionkey, r_name, r_comment, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ r_regionkey, r_name, r_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ r_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 11 { columns: [ n_nationkey, n_name, n_regionkey, _row_id, _rw_timestamp ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 12 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ n_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 13 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 14 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 14 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 15 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 15 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 16 - ├── columns: [ n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id_0, n_nationkey, s_suppkey, _row_id_1, _row_id_2, p_partkey ] + ├── columns: [ n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id_0, n_nationkey, s_suppkey, _row_id_1, _row_id_2, p_partkey, _rw_timestamp ] ├── primary key: [ $1 ASC, $4 ASC, $5 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 1 - Table 17 { columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, p_partkey, s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, p_partkey, s_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ o_orderkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 19 { columns: [ o_orderkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 20 { columns: [ n_name, s_suppkey, _row_id, n_nationkey, _row_id_0 ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 20 { columns: [ n_name, s_suppkey, _row_id, n_nationkey, _row_id_0, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 21 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id_0 ], primary key: [ $1 ASC, $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 22 { columns: [ l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id_0, _rw_timestamp ], primary key: [ $1 ASC, $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 23 { columns: [ l_suppkey, _row_id, _row_id_0, p_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 23 { columns: [ l_suppkey, _row_id, _row_id_0, p_partkey, _degree, _rw_timestamp ], 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: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ n_nationkey, n_name, n_regionkey, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ n_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 25 { columns: [ n_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 26 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 27 { columns: [ s_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 27 { columns: [ s_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 28 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 28 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 29 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ p_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 30 { columns: [ p_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 31 - ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id ] + ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC, $16 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 1 - Table 32 { columns: [ l_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 32 { columns: [ l_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 33 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 34 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 34 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 35 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 35 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], 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 } + Table 4294967294 { columns: [ o_year, mkt_share, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q9 before: @@ -1500,71 +1515,71 @@ └── StreamRowIdGen { row_id_index: 16 } └── StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } { tables: [ Source: 26 ] } - Table 0 { columns: [ n_name, $expr1, sum($expr2), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ n_name, $expr1, sum($expr2), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 1 { columns: [ p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id_0 ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 4 } + Table 1 { columns: [ p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 4 } - Table 2 { columns: [ p_partkey, ps_suppkey, ps_partkey, ps_suppkey_0, _row_id, _row_id_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 3 ], read pk prefix len hint: 4 } + Table 2 { columns: [ p_partkey, ps_suppkey, ps_partkey, ps_suppkey_0, _row_id, _row_id_0, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 3 ], read pk prefix len hint: 4 } Table 3 - ├── columns: [ n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey ] + ├── columns: [ n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, _rw_timestamp ] ├── primary key: [ $3 ASC, $4 ASC, $3 ASC, $1 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 4 - Table 4 { columns: [ l_partkey, l_suppkey, l_partkey_0, s_suppkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 3 ], read pk prefix len hint: 4 } + Table 4 { columns: [ l_partkey, l_suppkey, l_partkey_0, s_suppkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 3 ], read pk prefix len hint: 4 } - Table 5 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ p_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ p_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ ps_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ ps_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 9 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 10 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 10 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 11 { columns: [ n_name, s_suppkey, _row_id, n_nationkey, _row_id_0 ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 11 { columns: [ n_name, s_suppkey, _row_id, n_nationkey, _row_id_0, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 12 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree, _rw_timestamp ], 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: [ o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id_0 ], primary key: [ $2 ASC, $6 ASC, $8 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 13 { columns: [ o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id_0, _rw_timestamp ], primary key: [ $2 ASC, $6 ASC, $8 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 14 { columns: [ l_suppkey, _row_id, _row_id_0, o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ l_suppkey, _row_id, _row_id_0, o_orderkey, _degree, _rw_timestamp ], 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: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ n_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ n_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 17 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 18 { columns: [ s_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ s_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 19 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 20 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 20 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 21 { columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ o_orderkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 22 { columns: [ o_orderkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 23 - ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id ] + ├── columns: [ l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $16 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 24 { columns: [ l_orderkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ l_orderkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 25 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 26 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 26 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], 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 } + Table 4294967294 { columns: [ nation, o_year, sum_profit, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: tpch_q20 before: @@ -1743,51 +1758,51 @@ └── StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └── tables: [ Source: 21 ] - Table 0 { columns: [ s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id_0 ], primary key: [ $0 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ s_suppkey, _row_id, _row_id_0, s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ s_suppkey, _row_id, _row_id_0, s_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ ps_suppkey, _row_id, ps_partkey, l_partkey, l_suppkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ ps_suppkey, _row_id, ps_partkey, l_partkey, l_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ ps_suppkey, _row_id, ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ ps_suppkey, _row_id, ps_partkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 4 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 5 { columns: [ s_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ s_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ n_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ n_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 9 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 9 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 10 { columns: [ ps_partkey, ps_suppkey, $expr1, _row_id ], primary key: [ $0 ASC, $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 10 { columns: [ ps_partkey, ps_suppkey, $expr1, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 11 { columns: [ ps_partkey, ps_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 11 { columns: [ ps_partkey, ps_suppkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 12 { columns: [ $expr2, l_partkey, l_suppkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } + Table 12 { columns: [ $expr2, l_partkey, l_suppkey, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } - Table 13 { columns: [ l_partkey, l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 13 { columns: [ l_partkey, l_suppkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 14 { columns: [ ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ ps_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ ps_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ p_partkey, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ p_partkey, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ p_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ p_partkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 18 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 19 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 19 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 20 { columns: [ l_partkey, l_suppkey, sum(l_quantity), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 20 { columns: [ l_partkey, l_suppkey, sum(l_quantity), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 21 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 21 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3, 4, 5 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3, 4, 5 ], read pk prefix len hint: 5 } - id: tpch_q21 before: @@ -2000,80 +2015,80 @@ └── StreamFilter { predicate: (l_receiptdate > l_commitdate) } └── StreamExchange NoShuffle from 9 - Table 0 { columns: [ s_name, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ s_name, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ s_name, l_orderkey, l_suppkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, s_suppkey ] + ├── columns: [ s_name, l_orderkey, l_suppkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, s_suppkey, _rw_timestamp ] ├── primary key: [ $1 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 1 Table 2 - ├── columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, s_suppkey, _degree ] + ├── columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, s_suppkey, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ] ├── value indices: [ 8 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ l_orderkey, l_suppkey, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ l_orderkey, l_suppkey, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ l_orderkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ l_orderkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 - ├── columns: [ s_name, l_orderkey, l_suppkey, _row_id, _row_id_0, n_nationkey, s_suppkey, _row_id_1, _row_id_2, o_orderkey ] + ├── columns: [ s_name, l_orderkey, l_suppkey, _row_id, _row_id_0, n_nationkey, s_suppkey, _row_id_1, _row_id_2, o_orderkey, _rw_timestamp ] ├── primary key: [ $1 ASC, $3 ASC, $4 ASC, $5 ASC, $7 ASC, $8 ASC, $9 ASC, $6 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 1 ] └── read pk prefix len hint: 1 Table 6 - ├── columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, s_suppkey, _degree ] + ├── columns: [ l_orderkey, _row_id, _row_id_0, n_nationkey, _row_id_1, _row_id_2, o_orderkey, s_suppkey, _degree, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ] ├── value indices: [ 8 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 7 { columns: [ l_orderkey, l_suppkey, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ l_orderkey, l_suppkey, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ l_orderkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ l_orderkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ s_suppkey, s_name, _row_id, n_nationkey, _row_id_0 ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ s_suppkey, s_name, _row_id, n_nationkey, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ s_suppkey, _row_id, _row_id_0, n_nationkey, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id_0 ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 11 { columns: [ l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id_0, _rw_timestamp ], primary key: [ $1 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 12 { columns: [ l_suppkey, _row_id, _row_id_0, o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ l_suppkey, _row_id, _row_id_0, o_orderkey, _degree, _rw_timestamp ], 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: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 13 { columns: [ n_nationkey, n_name, n_regionkey, n_comment, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ n_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ n_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 15 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _rw_timestamp ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 16 { columns: [ s_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ s_nationkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 17 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 17 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 18 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 18 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 19 - ├── columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id ] + ├── columns: [ o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id, _rw_timestamp ] ├── primary key: [ $0 ASC, $9 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 20 { columns: [ o_orderkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ o_orderkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ l_orderkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 22 { columns: [ l_orderkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 23 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 24 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 24 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_name, numwait ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } diff --git a/src/frontend/planner_test/tests/testdata/output/union.yaml b/src/frontend/planner_test/tests/testdata/output/union.yaml index 57c65de3cd1f2..559b860be141d 100644 --- a/src/frontend/planner_test/tests/testdata/output/union.yaml +++ b/src/frontend/planner_test/tests/testdata/output/union.yaml @@ -41,7 +41,7 @@ └── BatchPlanNode Table 0 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -49,14 +49,19 @@ └── vnode column idx: 0 Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a, b, c, t1._row_id, $src ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3, 4 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ a, b, c, t1._row_id, $src, _rw_timestamp ] + ├── primary key: [ $3 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 3, 4 ] + └── read pk prefix len hint: 2 - sql: | create table t1 (a int, b numeric, c bigint); @@ -115,10 +120,10 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t1_a, t1_b, t1_c, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ t1_a, t1_b, t1_c, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -126,14 +131,14 @@ └── vnode column idx: 0 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ 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 } + Table 4294967294 { columns: [ a, b, c, _rw_timestamp ], 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: | create table t1 (a int, b numeric, c bigint, primary key(a)); @@ -193,14 +198,14 @@ └── BatchPlanNode Table 0 - ├── columns: [ t1_a, t1_b, t1_c, count ] + ├── columns: [ t1_a, t1_b, t1_c, count, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 Table 1 - ├── columns: [ vnode, a, backfill_finished, row_count ] + ├── columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -208,7 +213,7 @@ └── vnode column idx: 0 Table 2 - ├── columns: [ vnode, a, backfill_finished, row_count ] + ├── columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -216,7 +221,7 @@ └── vnode column idx: 0 Table 4294967294 - ├── columns: [ a, b, c ] + ├── columns: [ a, b, c, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] @@ -375,7 +380,7 @@ └── BatchPlanNode Table 0 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -383,7 +388,7 @@ └── vnode column idx: 0 Table 1 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -391,7 +396,7 @@ └── vnode column idx: 0 Table 2 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -399,7 +404,7 @@ └── vnode column idx: 0 Table 3 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -407,14 +412,19 @@ └── vnode column idx: 0 Table 4 - ├── columns: [ vnode, _row_id, backfill_finished, row_count ] + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] ├── 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: [ a, b, c, t1._row_id, $src ], primary key: [ $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3, 4 ], read pk prefix len hint: 2 } + Table 4294967294 + ├── columns: [ a, b, c, t1._row_id, $src, _rw_timestamp ] + ├── primary key: [ $3 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 3, 4 ] + └── read pk prefix len hint: 2 - name: test merged union stream key (5 columns, row_id + src_col + a + b + c) sql: | @@ -465,17 +475,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, a, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ], 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, b, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, b, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 2 { columns: [ vnode, c, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 { columns: [ vnode, c, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4 { columns: [ vnode, a, b, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, a, b, backfill_finished, row_count, _rw_timestamp ], 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: [ a, b, c, t1.a, null:Int64, null:Decimal, null:Serial, $src ], primary key: [ $3 ASC, $5 ASC, $4 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3, 5, 4, 6, 7 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ a, b, c, t1.a, null:Int64, null:Decimal, null:Serial, $src, _rw_timestamp ], primary key: [ $3 ASC, $5 ASC, $4 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3, 5, 4, 6, 7 ], read pk prefix len hint: 5 } - name: test merged union stream key (4 columns, row_id + src_col + a + b) sql: | @@ -526,17 +536,22 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, a, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 { columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ], 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, b, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 { columns: [ vnode, b, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 2 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4 { columns: [ vnode, a, b, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 { columns: [ vnode, a, b, backfill_finished, row_count, _rw_timestamp ], 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: [ a, b, c, t1.a, null:Decimal, null:Serial, $src ], primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 4, 5, 6 ], read pk prefix len hint: 4 } + Table 4294967294 + ├── columns: [ a, b, c, t1.a, null:Decimal, null:Serial, $src, _rw_timestamp ] + ├── primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 3, 4, 5, 6 ] + └── read pk prefix len hint: 4 - name: test merged union stream key (3 columns, src_col + a + b) sql: | @@ -593,7 +608,7 @@ └── BatchPlanNode Table 0 - ├── columns: [ vnode, a, backfill_finished, row_count ] + ├── columns: [ vnode, a, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -601,7 +616,7 @@ └── vnode column idx: 0 Table 1 - ├── columns: [ vnode, b, backfill_finished, row_count ] + ├── columns: [ vnode, b, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -609,7 +624,7 @@ └── vnode column idx: 0 Table 2 - ├── columns: [ vnode, b, backfill_finished, row_count ] + ├── columns: [ vnode, b, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -617,7 +632,7 @@ └── vnode column idx: 0 Table 3 - ├── columns: [ vnode, b, a, backfill_finished, row_count ] + ├── columns: [ vnode, b, a, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4 ] ├── distribution key: [ 0 ] @@ -625,7 +640,7 @@ └── vnode column idx: 0 Table 4 - ├── columns: [ vnode, a, b, backfill_finished, row_count ] + ├── columns: [ vnode, a, b, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4 ] ├── distribution key: [ 0 ] @@ -633,7 +648,7 @@ └── vnode column idx: 0 Table 4294967294 - ├── columns: [ a, b, c, $src ] + ├── columns: [ a, b, c, $src, _rw_timestamp ] ├── primary key: [ $0 ASC, $1 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0, 1, 3 ] @@ -714,16 +729,34 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ t1_a, t1_b, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ t1_a, t1_b, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 1 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 - Table 2 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 2 + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 - Table 3 { columns: [ vnode, _row_id, backfill_finished, row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 3 + ├── columns: [ vnode, _row_id, backfill_finished, row_count, _rw_timestamp ] + ├── 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: [ b, t1.a, t1.b, null:Serial, $src ] + ├── columns: [ b, t1.a, t1.b, null:Serial, $src, _rw_timestamp ] ├── primary key: [ $1 ASC, $2 ASC, $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 1, 2, 3, 4 ] diff --git a/src/frontend/planner_test/tests/testdata/output/update.yaml b/src/frontend/planner_test/tests/testdata/output/update.yaml index 884b091cf8af7..6e26252840629 100644 --- a/src/frontend/planner_test/tests/testdata/output/update.yaml +++ b/src/frontend/planner_test/tests/testdata/output/update.yaml @@ -6,7 +6,7 @@ BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [0:Int32, $1, $2] } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int, v2 int); update t set v1 = true; @@ -18,7 +18,7 @@ BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [($1 + 1:Int32), $1, $2] } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int, v2 real); update t set v1 = v2; @@ -26,7 +26,7 @@ BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [$1::Int32, $1, $2] } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int, v2 real); update t set v1 = DEFAULT; @@ -34,7 +34,7 @@ BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [null:Int32, $1, $2] } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int, v2 int); update t set v1 = v2 + 1 where v2 > 0; @@ -43,7 +43,7 @@ └─BatchUpdate { table: t, exprs: [($1 + 1:Int32), $1, $2] } └─BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (t.v2 > 0:Int32) } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int, v2 int); update t set (v1, v2) = (v2 + 1, v1 - 1) where v1 != v2; @@ -52,7 +52,7 @@ └─BatchUpdate { table: t, exprs: [($1 + 1:Int32), ($0 - 1:Int32), $2] } └─BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (t.v1 <> t.v2) } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int, v2 int); update t set (v1, v2) = (v2 + 1, v1 - 1) where v1 != v2 returning *, v2+1, v1-1; @@ -60,14 +60,14 @@ LogicalProject { exprs: [t.v1, t.v2, (t.v2 + 1:Int32) as $expr1, (t.v1 - 1:Int32) as $expr2] } └─LogicalUpdate { table: t, exprs: [($1 + 1:Int32), ($0 - 1:Int32), $2], returning: true } └─LogicalFilter { predicate: (t.v1 <> t.v2) } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v1, t.v2, (t.v2 + 1:Int32) as $expr1, (t.v1 - 1:Int32) as $expr2] } └─BatchUpdate { table: t, exprs: [($1 + 1:Int32), ($0 - 1:Int32), $2], returning: true } └─BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (t.v1 <> t.v2) } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - name: update with returning statement, should keep `Update` sql: | create table t (v int); @@ -75,13 +75,13 @@ logical_plan: |- LogicalProject { exprs: [514:Int32] } └─LogicalUpdate { table: t, exprs: [114:Int32, $1], returning: true } - └─LogicalScan { table: t, columns: [t.v, t._row_id] } + └─LogicalScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [514:Int32] } └─BatchUpdate { table: t, exprs: [114:Int32, $1], returning: true } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: t, columns: [t.v, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int primary key, v2 int); update t set (v2, v1) = (v1, v2); @@ -91,12 +91,12 @@ update t set v1 = default; logical_plan: |- LogicalUpdate { table: t, exprs: [(1:Int32 + 1:Int32), $1, $2] } - └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [2:Int32, $1, $2] } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - name: update table with generated columns sql: | create table t(v1 int as v2-1, v2 int, v3 int as v2+1); @@ -105,7 +105,7 @@ BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [3:Int32, $3] } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } - name: update generated column sql: | create table t(v1 int as v2-1, v2 int, v3 int as v2+1); @@ -123,16 +123,16 @@ logical_plan: |- LogicalUpdate { table: t, exprs: [777:Int32, $1, $2] } └─LogicalApply { type: LeftAnti, on: (t.b = t.a), correlated_id: 1 } - ├─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + ├─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } └─LogicalProject { exprs: [t.a] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [777:Int32, $1, $2] } └─BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: LeftAnti, predicate: t.b = t.a, output: all } ├─BatchExchange { order: [], dist: HashShard(t.b) } - │ └─BatchScan { table: t, columns: [t.a, t.b, t._row_id], distribution: UpstreamHashShard(t._row_id) } + │ └─BatchScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } └─BatchExchange { order: [], dist: HashShard(t.a) } └─BatchScan { table: t, columns: [t.a], distribution: SomeShard } - name: delete subquery @@ -141,10 +141,11 @@ delete from t where a not in (select b from t); logical_plan: |- LogicalDelete { table: t } - └─LogicalApply { type: LeftAnti, on: (t.a = t.b), correlated_id: 1 } - ├─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } - └─LogicalProject { exprs: [t.b] } - └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } + └─LogicalProject { exprs: [t.a, t.b, t._row_id] } + └─LogicalApply { type: LeftAnti, on: (t.a = t.b), correlated_id: 1 } + ├─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } + └─LogicalProject { exprs: [t.b] } + └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchDelete { table: t } @@ -163,5 +164,5 @@ BatchSimpleAgg { aggs: [sum()] } └─BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [($0 + 1:Int32), $1, $2] } - └─BatchExchange { order: [], dist: HashShard(t.a, t.b, t._row_id) } - └─BatchScan { table: t, columns: [t.a, t.b, t._row_id], distribution: UpstreamHashShard(t._row_id) } + └─BatchExchange { order: [], dist: HashShard(t.a, t.b, t._row_id, t._rw_timestamp) } + └─BatchScan { table: t, columns: [t.a, t.b, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index 67e098154ece6..8949c5382e888 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -23,7 +23,7 @@ └── StreamSource { source: t, columns: [v1, _row_id] } { tables: [ Source: 1 ] } Table 0 - ├── columns: [ vnode, offset ] + ├── columns: [ vnode, offset, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1 ] ├── distribution key: [ 0 ] @@ -31,14 +31,14 @@ └── vnode column idx: 0 Table 1 - ├── columns: [ partition_id, offset_info ] + ├── columns: [ partition_id, offset_info, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1 ] ├── distribution key: [] └── read pk prefix len hint: 1 Table 4294967294 - ├── columns: [ v1, _row_id ] + ├── columns: [ v1, _row_id, _rw_timestamp ] ├── primary key: [ $1 ASC ] ├── value indices: [ 0, 1 ] ├── distribution key: [ 1 ] @@ -112,8 +112,8 @@ logical_plan: |- LogicalProject { exprs: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2] } └─LogicalJoin { type: LeftOuter, on: (t1.v1 = t2.v1) AND (t1.ts >= (t2.ts + '00:00:01':Interval)) AND (t2.ts >= (t1.ts + '00:00:01':Interval)), output: all } - ├─LogicalScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id, t2._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] } └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } @@ -132,8 +132,8 @@ logical_plan: |- LogicalProject { exprs: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v1) AND (t1.ts >= (t2.ts + '00:00:01':Interval)) AND (t2.ts >= (t1.ts + '00:00:01':Interval)), output: all } - ├─LogicalScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id] } - └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] } + ├─LogicalScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id, t1._rw_timestamp] } + └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id, t2._rw_timestamp] } stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] } └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 85ed93c7dc0ca..86631767998bd 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -983,6 +983,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { description: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, }) } diff --git a/src/frontend/src/binder/update.rs b/src/frontend/src/binder/update.rs index cbe51f9ec69bd..2f111c0756416 100644 --- a/src/frontend/src/binder/update.rs +++ b/src/frontend/src/binder/update.rs @@ -219,7 +219,8 @@ impl Binder { .iter() .enumerate() .filter_map(|(i, c)| { - (!c.is_generated()).then_some(InputRef::new(i, c.data_type().clone()).into()) + c.can_dml() + .then_some(InputRef::new(i, c.data_type().clone()).into()) }) .map(|c| assignment_exprs.remove(&c).unwrap_or(c)) .collect_vec(); diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index b533e6d956685..3c13715a63f28 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -328,6 +328,14 @@ impl TableCatalog { &self.columns } + pub fn columns_without_rw_timestamp(&self) -> Vec { + self.columns + .iter() + .filter(|c| !c.is_rw_timestamp_column()) + .cloned() + .collect() + } + /// Get a reference to the table catalog's pk desc. pub fn pk(&self) -> &[ColumnOrder] { self.pk.as_ref() @@ -412,7 +420,12 @@ impl TableCatalog { schema_id, database_id, name: self.name.clone(), - columns: self.columns().iter().map(|c| c.to_protobuf()).collect(), + // ignore `_rw_timestamp` when serializing + columns: self + .columns_without_rw_timestamp() + .iter() + .map(|c| c.to_protobuf()) + .collect(), pk: self.pk.iter().map(|o| o.to_protobuf()).collect(), stream_key: self.stream_key.iter().map(|x| *x as _).collect(), dependent_relations: vec![], @@ -530,6 +543,10 @@ impl TableCatalog { self.columns.iter().any(|c| c.is_generated()) } + pub fn has_rw_timestamp_column(&self) -> bool { + self.columns.iter().any(|c| c.is_rw_timestamp_column()) + } + pub fn column_schema(&self) -> Schema { Schema::new( self.columns @@ -570,7 +587,12 @@ impl From for TableCatalog { let conflict_behavior = ConflictBehavior::from_protobuf(&tb_conflict_behavior); let version_column_index = tb.version_column_index.map(|value| value as usize); - let columns: Vec = tb.columns.into_iter().map(ColumnCatalog::from).collect(); + let mut columns: Vec = + tb.columns.into_iter().map(ColumnCatalog::from).collect(); + if columns.iter().all(|c| !c.is_rw_timestamp_column()) { + // Add system column `_rw_timestamp` to every table, but notice that this column is never persisted. + columns.push(ColumnCatalog::rw_timestamp_column()); + } for (idx, catalog) in columns.clone().into_iter().enumerate() { let col_name = catalog.name(); if !col_names.insert(col_name.to_string()) { @@ -755,9 +777,11 @@ mod tests { generated_or_default_column: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, }, is_hidden: false - } + }, + ColumnCatalog::rw_timestamp_column(), ], stream_key: vec![0], pk: vec![ColumnOrder::new(0, OrderType::ascending())], @@ -774,7 +798,7 @@ mod tests { conflict_behavior: ConflictBehavior::NoCheck, read_prefix_len_hint: 0, version: Some(TableVersion::new_initial_for_test(ColumnId::new(1))), - watermark_columns: FixedBitSet::with_capacity(2), + watermark_columns: FixedBitSet::with_capacity(3), dist_key_in_pk: vec![], cardinality: Cardinality::unknown(), created_at_epoch: None, diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 5ba3c804ee748..2bbd9af648931 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -233,6 +233,7 @@ pub async fn get_replace_table_plan( .columns .iter() .map(|col| ColumnCatalog::from(col.clone())) + .filter(|col| !col.is_rw_timestamp_column()) .collect_vec(); for sink in fetch_incoming_sinks(session, &incoming_sink_ids)? { diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 1c8a866db3e06..2c913a6834d37 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -275,7 +275,9 @@ pub mod tests { use std::collections::HashMap; use pgwire::pg_response::StatementType::CREATE_MATERIALIZED_VIEW; - use risingwave_common::catalog::{DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, ROWID_PREFIX}; + use risingwave_common::catalog::{ + DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, ROWID_PREFIX, RW_TIMESTAMP_COLUMN_NAME, + }; use risingwave_common::types::DataType; use crate::catalog::root_catalog::SchemaPath; @@ -327,7 +329,8 @@ pub mod tests { "country" => DataType::new_struct( vec![DataType::Varchar,city_type,DataType::Varchar], vec!["address".to_string(), "city".to_string(), "zipcode".to_string()], - ) + ), + RW_TIMESTAMP_COLUMN_NAME => DataType::Timestamptz, }; assert_eq!(columns, expected_columns); } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index c516e47fc7ce3..db6c1f14d1ae3 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -272,14 +272,17 @@ pub async fn gen_sink_plan( if let Some(table_catalog) = &target_table_catalog { for column in sink_catalog.full_columns() { - if column.is_generated() { - unreachable!("can not derive generated columns in a sink's catalog, but meet one"); + if !column.can_dml() { + unreachable!("can not derive generated columns and system column `_rw_timestamp` in a sink's catalog, but meet one"); } } + + let table_columns_without_rw_timestamp = table_catalog.columns_without_rw_timestamp(); + let exprs = derive_default_column_project_for_sink( &sink_catalog, sink_plan.schema(), - table_catalog.columns(), + &table_columns_without_rw_timestamp, user_specified_columns, )?; @@ -287,8 +290,9 @@ pub async fn gen_sink_plan( sink_plan = StreamProject::new(logical_project).into(); - let exprs = - LogicalSource::derive_output_exprs_from_generated_columns(table_catalog.columns())?; + let exprs = LogicalSource::derive_output_exprs_from_generated_columns( + &table_columns_without_rw_timestamp, + )?; if let Some(exprs) = exprs { let logical_project = generic::Project::new(exprs, sink_plan); @@ -466,17 +470,18 @@ pub async fn handle_create_sink( let incoming_sink_ids: HashSet<_> = table_catalog.incoming_sinks.iter().copied().collect(); let incoming_sinks = fetch_incoming_sinks(&session, &incoming_sink_ids)?; + let columns_without_rw_timestamp = table_catalog.columns_without_rw_timestamp(); for existing_sink in incoming_sinks { hijack_merger_for_target_table( &mut graph, - table_catalog.columns(), + &columns_without_rw_timestamp, &existing_sink, Some(&existing_sink.unique_identity()), )?; } // for new creating sink, we don't have a unique identity because the sink id is not generated yet. - hijack_merger_for_target_table(&mut graph, table_catalog.columns(), &sink, None)?; + hijack_merger_for_target_table(&mut graph, &columns_without_rw_timestamp, &sink, None)?; target_table_replace_plan = Some(ReplaceTablePlan { source, @@ -772,7 +777,7 @@ pub(crate) fn derive_default_column_project_for_sink( .collect::>(); for (idx, column) in columns.iter().enumerate() { - if column.is_generated() { + if !column.can_dml() { continue; } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 73fb65dbf3a49..b049c1b33708c 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -216,6 +216,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> description: None, additional_column: AdditionalColumn { column_type: None }, version: ColumnDescVersion::Pr13707, + system_column: None, }, is_hidden: false, }); @@ -1484,7 +1485,9 @@ fn get_source_and_resolved_table_name( #[cfg(test)] mod tests { - use risingwave_common::catalog::{Field, DEFAULT_DATABASE_NAME, ROWID_PREFIX}; + use risingwave_common::catalog::{ + Field, DEFAULT_DATABASE_NAME, ROWID_PREFIX, RW_TIMESTAMP_COLUMN_NAME, + }; use risingwave_common::types::DataType; use super::*; @@ -1554,6 +1557,7 @@ mod tests { vec![DataType::Int64,DataType::Float64,DataType::Float64], vec!["v3".to_string(), "v4".to_string(), "v5".to_string()], ), + RW_TIMESTAMP_COLUMN_NAME => DataType::Timestamptz, }; assert_eq!(columns, expected_columns); diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 0cb2bf02304a7..dce011c3f3d24 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -282,6 +282,7 @@ mod tests { "v4".into() => "integer".into(), "primary key".into() => "v3".into(), "distribution key".into() => "v3".into(), + "_rw_timestamp".into() => "timestamp with time zone".into(), "idx1".into() => "index(v1 DESC, v2 ASC, v3 ASC) include(v4) distributed by(v1)".into(), "table description".into() => "t".into(), }; diff --git a/src/frontend/src/handler/drop_sink.rs b/src/frontend/src/handler/drop_sink.rs index 967fe700dfcde..8e0d015d0f632 100644 --- a/src/frontend/src/handler/drop_sink.rs +++ b/src/frontend/src/handler/drop_sink.rs @@ -84,10 +84,11 @@ pub async fn handle_drop_sink( assert!(incoming_sink_ids.remove(&sink_id.sink_id)); + let columns_without_rw_timestamp = table_catalog.columns_without_rw_timestamp(); for sink in fetch_incoming_sinks(&session, &incoming_sink_ids)? { hijack_merger_for_target_table( &mut graph, - table_catalog.columns(), + &columns_without_rw_timestamp, &sink, Some(&sink.unique_identity()), )?; diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index b91270d0607ea..3fcdac7461f94 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -81,7 +81,7 @@ use crate::optimizer::plan_node::{ BatchExchange, PlanNodeType, PlanTreeNode, RewriteExprsRecursive, StreamExchange, StreamUnion, ToStream, VisitExprsRecursive, }; -use crate::optimizer::plan_visitor::TemporalJoinValidator; +use crate::optimizer::plan_visitor::{RwTimestampValidator, TemporalJoinValidator}; use crate::optimizer::property::Distribution; use crate::utils::{ColIndexMappingRewriteExt, WithOptionsSecResolved}; use crate::TableCatalog; @@ -523,6 +523,13 @@ impl PlanRoot { ).into()); } + if RwTimestampValidator::select_rw_timestamp_in_stream_query(plan.clone()) { + return Err(ErrorCode::NotSupported( + "selecting `_rw_timestamp` in a streaming query is not allowed".to_string(), + "please run the sql in batch mode or remove the column `_rw_timestamp` from the streaming query".to_string(), + ).into()); + } + self.plan = plan; self.phase = PlanPhase::Stream; assert_eq!(self.plan.convention(), Convention::Stream); @@ -724,7 +731,7 @@ impl PlanRoot { let column_descs = columns .iter() - .filter(|&c| (!c.is_generated())) + .filter(|&c| c.can_dml()) .map(|c| c.column_desc.clone()) .collect(); diff --git a/src/frontend/src/optimizer/plan_visitor/mod.rs b/src/frontend/src/optimizer/plan_visitor/mod.rs index 632d0bcf8bd9d..bc6c3141cad2e 100644 --- a/src/frontend/src/optimizer/plan_visitor/mod.rs +++ b/src/frontend/src/optimizer/plan_visitor/mod.rs @@ -41,8 +41,10 @@ mod jsonb_stream_key_checker; pub use jsonb_stream_key_checker::*; mod distributed_dml_visitor; mod read_storage_table_visitor; +mod rw_timestamp_validator; pub use distributed_dml_visitor::*; pub use read_storage_table_visitor::*; +pub use rw_timestamp_validator::*; use crate::for_all_plan_nodes; use crate::optimizer::plan_node::*; diff --git a/src/frontend/src/optimizer/plan_visitor/rw_timestamp_validator.rs b/src/frontend/src/optimizer/plan_visitor/rw_timestamp_validator.rs new file mode 100644 index 0000000000000..ec1fbc867e5ce --- /dev/null +++ b/src/frontend/src/optimizer/plan_visitor/rw_timestamp_validator.rs @@ -0,0 +1,47 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_common::catalog::{RW_TIMESTAMP_COLUMN_ID, RW_TIMESTAMP_COLUMN_NAME}; + +use super::{DefaultBehavior, Merge}; +use crate::optimizer::plan_node::StreamTableScan; +use crate::optimizer::plan_visitor::PlanVisitor; +use crate::PlanRef; + +#[derive(Debug, Clone, Default)] +pub struct RwTimestampValidator {} + +impl RwTimestampValidator { + pub fn select_rw_timestamp_in_stream_query(plan: PlanRef) -> bool { + RwTimestampValidator::default().visit(plan) + } +} + +impl PlanVisitor for RwTimestampValidator { + type Result = bool; + + type DefaultBehavior = impl DefaultBehavior; + + fn default_behavior() -> Self::DefaultBehavior { + Merge(|a, b| a | b) + } + + fn visit_stream_table_scan(&mut self, stream_table_scan: &StreamTableScan) -> bool { + stream_table_scan + .core() + .column_descs() + .iter() + .any(|c| c.column_id == RW_TIMESTAMP_COLUMN_ID && c.name == RW_TIMESTAMP_COLUMN_NAME) + } +} diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs index a995dd9878620..c968362853959 100644 --- a/src/frontend/src/optimizer/rule/index_selection_rule.rs +++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs @@ -214,15 +214,6 @@ impl IndexSelectionRule { // 1. logical_scan -> logical_join // / \ // index_scan primary_table_scan - let predicate = logical_scan.predicate().clone(); - let offset = index.index_item.len(); - let mut rewriter = IndexPredicateRewriter::new( - index.primary_to_secondary_mapping(), - index.function_mapping(), - offset, - ); - let new_predicate = predicate.rewrite_expr(&mut rewriter); - let index_scan = LogicalScan::create( index.index_table.name.clone(), index.index_table.clone(), @@ -231,6 +222,9 @@ impl IndexSelectionRule { logical_scan.as_of().clone(), index.index_table.cardinality, ); + // We use `schema.len` instead of `index_item.len` here, + // because schema contains system columns like `_rw_timestamp` column which is not represented in the index item. + let offset = index_scan.table_catalog().columns().len(); let primary_table_scan = LogicalScan::create( index.primary_table.name.clone(), @@ -241,6 +235,14 @@ impl IndexSelectionRule { index.primary_table.cardinality, ); + let predicate = logical_scan.predicate().clone(); + let mut rewriter = IndexPredicateRewriter::new( + index.primary_to_secondary_mapping(), + index.function_mapping(), + offset, + ); + let new_predicate = predicate.rewrite_expr(&mut rewriter); + let conjunctions = index .primary_table_pk_ref_to_index_table() .iter() @@ -251,7 +253,7 @@ impl IndexSelectionRule { index.index_table.columns[x.column_index] .data_type() .clone(), - y.column_index + index.index_item.len(), + y.column_index + offset, index.primary_table.columns[y.column_index] .data_type() .clone(), diff --git a/src/frontend/src/planner/delete.rs b/src/frontend/src/planner/delete.rs index c4c34346ec1d2..4cdca1cc7f105 100644 --- a/src/frontend/src/planner/delete.rs +++ b/src/frontend/src/planner/delete.rs @@ -29,7 +29,9 @@ impl Planner { } else { scan }; - let input = if delete.table.table_catalog.has_generated_column() { + let input = if delete.table.table_catalog.has_generated_column() + || delete.table.table_catalog.has_rw_timestamp_column() + { LogicalProject::with_out_col_idx( input, delete @@ -38,7 +40,7 @@ impl Planner { .columns() .iter() .enumerate() - .filter_map(|(i, c)| (!c.is_generated()).then_some(i)), + .filter_map(|(i, c)| (c.can_dml()).then_some(i)), ) .into() } else { diff --git a/src/frontend/src/planner/update.rs b/src/frontend/src/planner/update.rs index ef735db1b5a95..ddf9ab0bdf9ae 100644 --- a/src/frontend/src/planner/update.rs +++ b/src/frontend/src/planner/update.rs @@ -37,7 +37,7 @@ impl Planner { .columns() .iter() .enumerate() - .filter_map(|(i, c)| (!c.is_generated()).then_some(i)) + .filter_map(|(i, c)| c.can_dml().then_some(i)) .collect_vec(); let mut plan: PlanRef = LogicalUpdate::from(generic::Update::new( diff --git a/src/storage/src/row_serde/mod.rs b/src/storage/src/row_serde/mod.rs index 138d2361468c5..d25d2f3ea9c7f 100644 --- a/src/storage/src/row_serde/mod.rs +++ b/src/storage/src/row_serde/mod.rs @@ -103,6 +103,7 @@ mod test { column_type: None, }, version: Pr13707, + system_column: None, }, ColumnDesc { data_type: Int16, @@ -116,6 +117,7 @@ mod test { column_type: None, }, version: Pr13707, + system_column: None, }, ], [ @@ -149,6 +151,7 @@ mod test { column_type: None, }, version: Pr13707, + system_column: None, }, ColumnDesc { data_type: Varchar, @@ -162,6 +165,7 @@ mod test { column_type: None, }, version: Pr13707, + system_column: None, }, ], [ diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 57fcf27c66486..48b9c86c0df2b 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -31,6 +31,7 @@ use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, TableId, TableOpt use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, OwnedRow, Row, RowExt}; use risingwave_common::types::ToOwnedDatum; +use risingwave_common::util::epoch::Epoch; use risingwave_common::util::row_serde::*; use risingwave_common::util::sort_util::OrderType; use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; @@ -86,6 +87,9 @@ pub struct StorageTableInner { /// Mapping from column id to column index for deserializing the row. mapping: Arc, + /// The index of system column `_rw_timestamp` in the output columns. + epoch_idx: Option, + /// Row deserializer to deserialize the value in storage to a row. /// The row can be either complete or partial, depending on whether the row encoding is versioned. row_serde: Arc, @@ -243,12 +247,17 @@ impl StorageTableInner { let mut value_output_indices = vec![]; let mut key_output_indices = vec![]; + // system column currently only contains `_rw_timestamp` + let mut epoch_idx = None; for idx in &output_indices { if value_indices.contains(idx) { value_output_indices.push(*idx); - } else { + } else if pk_indices.contains(idx) { key_output_indices.push(*idx); + } else { + assert!(epoch_idx.is_none()); + epoch_idx = Some(*idx); } } @@ -308,6 +317,7 @@ impl StorageTableInner { value_output_indices, output_row_in_key_indices, mapping: Arc::new(mapping), + epoch_idx, row_serde: Arc::new(row_serde), pk_indices, distribution, @@ -360,6 +370,8 @@ impl StorageTableInner { pk: impl Row, wait_epoch: HummockReadEpoch, ) -> StorageResult> { + // `get_row` doesn't support select `_rw_timestamp` yet. + assert!(self.epoch_idx.is_none()); let epoch = wait_epoch.get_epoch(); let read_backup = matches!(wait_epoch, HummockReadEpoch::Backup(_)); let read_committed = wait_epoch.is_read_committed(); @@ -440,6 +452,10 @@ impl StorageTableInner { pub fn update_vnode_bitmap(&mut self, new_vnodes: Arc) -> Arc { self.distribution.update_vnode_bitmap(new_vnodes) } + + pub fn has_epoch_idx(&self) -> bool { + self.epoch_idx.is_some() + } } pub trait PkAndRowStream = Stream>> + Send; @@ -513,6 +529,7 @@ impl StorageTableInner { let iter = StorageTableInnerIterInner::::new( &self.store, self.mapping.clone(), + self.epoch_idx, pk_serializer, self.output_indices.clone(), self.key_output_indices.clone(), @@ -845,6 +862,9 @@ struct StorageTableInnerIterInner { mapping: Arc, + /// The index of system column `_rw_timestamp` in the output columns. + epoch_idx: Option, + row_deserializer: Arc, /// Used for serializing and deserializing the primary key. @@ -868,6 +888,7 @@ impl StorageTableInnerIterInner { async fn new( store: &S, mapping: Arc, + epoch_idx: Option, pk_serializer: Option>, output_indices: Vec, key_output_indices: Option>, @@ -891,6 +912,7 @@ impl StorageTableInnerIterInner { let iter = Self { iter, mapping, + epoch_idx, row_deserializer, pk_serializer, output_indices, @@ -910,7 +932,7 @@ impl StorageTableInnerIterInner { .verbose_instrument_await("storage_table_iter_next") .await? { - let (table_key, value) = (k.user_key.table_key, v); + let (table_key, value, epoch_with_gap) = (k.user_key.table_key, v, k.epoch_with_gap); let row = self.row_deserializer.deserialize(value)?; let result_row_in_value = self.mapping.project(OwnedRow::new(row)); match &self.key_output_indices { @@ -926,7 +948,13 @@ impl StorageTableInnerIterInner { let mut result_row_vec = vec![]; for idx in &self.output_indices { - if self.value_output_indices.contains(idx) { + if let Some(epoch_idx) = self.epoch_idx + && *idx == epoch_idx + { + let epoch = Epoch::from(epoch_with_gap.pure_epoch()); + result_row_vec + .push(risingwave_common::types::Datum::from(epoch.as_scalar())); + } else if self.value_output_indices.contains(idx) { let item_position_in_value_indices = &self .value_output_indices .iter() @@ -955,12 +983,40 @@ impl StorageTableInnerIterInner { row, } } - None => { - yield KeyedRow { - vnode_prefixed_key: table_key.copy_into(), - row: result_row_in_value.into_owned_row(), + None => match &self.epoch_idx { + Some(epoch_idx) => { + let mut result_row_vec = vec![]; + for idx in &self.output_indices { + if idx == epoch_idx { + let epoch = Epoch::from(epoch_with_gap.pure_epoch()); + result_row_vec + .push(risingwave_common::types::Datum::from(epoch.as_scalar())); + } else { + let item_position_in_value_indices = &self + .value_output_indices + .iter() + .position(|p| idx == p) + .unwrap(); + result_row_vec.push( + result_row_in_value + .datum_at(*item_position_in_value_indices) + .to_owned_datum(), + ); + } + } + let row = OwnedRow::new(result_row_vec); + yield KeyedRow { + vnode_prefixed_key: table_key.copy_into(), + row, + } } - } + None => { + yield KeyedRow { + vnode_prefixed_key: table_key.copy_into(), + row: result_row_in_value.into_owned_row(), + } + } + }, } } }