From a644ce9b581bffa522e1fddebe2b913da942b87a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 28 Oct 2024 19:29:14 +0800 Subject: [PATCH 01/42] match rw predicates --- .../plan_node/logical_iceberg_scan.rs | 95 +++++++++++++++++++ 1 file changed, 95 insertions(+) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index b355d6f2057c8..31158ff1b0237 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -14,7 +14,9 @@ use std::rc::Rc; +use iceberg::expr::Predicate as IcebergPredicate; use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::types::ScalarImpl; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; @@ -24,6 +26,7 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; +use crate::expr::{ExprImpl, ExprType}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ @@ -102,11 +105,103 @@ impl ExprRewritable for LogicalIcebergScan {} impl ExprVisitable for LogicalIcebergScan {} impl PredicatePushdown for LogicalIcebergScan { + /// NOTE(kwannoel): + /// 1. We expect it to be constant folded + /// 2. We don't convert `inputRefs` of type boolean directly to IcebergPredicates. + /// 3. The leaf nodes are always logical comparison operators: + /// `Equal`, `NotEqual`, `GreaterThan`, + /// `GreaterThanOrEqual`, `LessThan`, `LessThanOrEqual`. + /// 4. For leaf nodes, their LHS is always an `inputRef` + /// and their RHS is always a `Literal` to be compatible with Iceberg. fn predicate_pushdown( &self, predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { + fn rw_expr_to_iceberg_predicate(expr: &ExprImpl) -> Option { + match expr { + ExprImpl::Literal(l) => match l.get_data() { + Some(ScalarImpl::Bool(b)) => { + if *b { + Some(IcebergPredicate::AlwaysTrue) + } else { + Some(IcebergPredicate::AlwaysFalse) + } + } + _ => None, + }, + ExprImpl::FunctionCall(f) => { + let args = f.inputs(); + match f.func_type() { + ExprType::Not => { + let arg = rw_expr_to_iceberg_predicate(&args[0])?; + Some(IcebergPredicate::negate(arg)) + } + ExprType::And => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0])?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1])?; + Some(IcebergPredicate::and(arg0, arg1)) + } + ExprType::Or => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0])?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1])?; + Some(IcebergPredicate::or(arg0, arg1)) + } + ExprType::Equal => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + todo!() + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + todo!() + } + _ => None, + }, + ExprType::NotEqual => { + todo!() + } + ExprType::GreaterThan => { + todo!() + } + ExprType::GreaterThanOrEqual => { + todo!() + } + ExprType::LessThan => { + todo!() + } + ExprType::LessThanOrEqual => { + todo!() + } + ExprType::IsNull => { + todo!() + } + ExprType::IsNotNull => { + todo!() + } + ExprType::In => { + todo!() + } + _ => None, + } + } + _ => None, + } + } + fn rw_predicate_to_iceberg_predicate(predicate: Condition) -> IcebergPredicate { + if predicate.always_true() { + return IcebergPredicate::AlwaysTrue; + } + let mut conjunctions = predicate.conjunctions; + let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); + let rw_condition_root = conjunctions.pop().unwrap(); + let iceberg_condition_root = rw_expr_to_iceberg_predicate(&rw_condition_root); + for rw_condition in conjunctions { + match rw_expr_to_iceberg_predicate(&rw_condition) { + Some(iceberg_predicate) => ignored_conjunctions.push(rw_condition), + None => {} + } + } + todo!() + } // No pushdown. LogicalFilter::create(self.clone().into(), predicate) } From 8783214f4d0a849ecc6ee01a7ee173663f1b467f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 16:54:59 +0800 Subject: [PATCH 02/42] mark place to add filter --- src/connector/src/source/iceberg/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 60a26e43e1d31..c6f5f08d1b305 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -249,6 +249,7 @@ impl IcebergSplitEnumerator { let mut position_delete_files = vec![]; let mut data_files = vec![]; let mut equality_delete_files = vec![]; + // TODO(kwannoel): add filter here. let scan = table .scan() .snapshot_id(snapshot_id) From fe92b995b58c622c6150e705b2deb9cde2048062 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 1 Nov 2024 15:49:05 +0800 Subject: [PATCH 03/42] pass in schema fields as a parameter --- .../plan_node/logical_iceberg_scan.rs | 28 +++++++++++++------ 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 31158ff1b0237..f2b4048b6f94d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -16,6 +16,7 @@ use std::rc::Rc; use iceberg::expr::Predicate as IcebergPredicate; use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::ScalarImpl; use super::generic::GenericPlanRef; @@ -118,7 +119,10 @@ impl PredicatePushdown for LogicalIcebergScan { predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { - fn rw_expr_to_iceberg_predicate(expr: &ExprImpl) -> Option { + fn rw_expr_to_iceberg_predicate( + expr: &ExprImpl, + fields: &[Field], + ) -> Option { match expr { ExprImpl::Literal(l) => match l.get_data() { Some(ScalarImpl::Bool(b)) => { @@ -134,17 +138,17 @@ impl PredicatePushdown for LogicalIcebergScan { let args = f.inputs(); match f.func_type() { ExprType::Not => { - let arg = rw_expr_to_iceberg_predicate(&args[0])?; + let arg = rw_expr_to_iceberg_predicate(&args[0], fields)?; Some(IcebergPredicate::negate(arg)) } ExprType::And => { - let arg0 = rw_expr_to_iceberg_predicate(&args[0])?; - let arg1 = rw_expr_to_iceberg_predicate(&args[1])?; + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; Some(IcebergPredicate::and(arg0, arg1)) } ExprType::Or => { - let arg0 = rw_expr_to_iceberg_predicate(&args[0])?; - let arg1 = rw_expr_to_iceberg_predicate(&args[1])?; + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; Some(IcebergPredicate::or(arg0, arg1)) } ExprType::Equal => match [&args[0], &args[1]] { @@ -186,22 +190,28 @@ impl PredicatePushdown for LogicalIcebergScan { _ => None, } } - fn rw_predicate_to_iceberg_predicate(predicate: Condition) -> IcebergPredicate { + fn rw_predicate_to_iceberg_predicate( + predicate: Condition, + fields: &[Field], + ) -> IcebergPredicate { if predicate.always_true() { return IcebergPredicate::AlwaysTrue; } let mut conjunctions = predicate.conjunctions; let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); let rw_condition_root = conjunctions.pop().unwrap(); - let iceberg_condition_root = rw_expr_to_iceberg_predicate(&rw_condition_root); + let iceberg_condition_root = rw_expr_to_iceberg_predicate(&rw_condition_root, fields); for rw_condition in conjunctions { - match rw_expr_to_iceberg_predicate(&rw_condition) { + match rw_expr_to_iceberg_predicate(&rw_condition, fields) { Some(iceberg_predicate) => ignored_conjunctions.push(rw_condition), None => {} } } todo!() } + + let schema = self.schema(); + let fields = &schema.fields; // No pushdown. LogicalFilter::create(self.clone().into(), predicate) } From 19645b7ed5a7c694d9c69e14d19e3ef5e20e9746 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 1 Nov 2024 16:48:02 +0800 Subject: [PATCH 04/42] convert input ref to reference, datum to iceberg datum --- .../plan_node/logical_iceberg_scan.rs | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index f2b4048b6f94d..f789994fa887f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -14,7 +14,8 @@ use std::rc::Rc; -use iceberg::expr::Predicate as IcebergPredicate; +use iceberg::expr::{Predicate as IcebergPredicate, Reference}; +use iceberg::spec::Datum as IcebergDatum; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::ScalarImpl; @@ -27,7 +28,7 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; -use crate::expr::{ExprImpl, ExprType}; +use crate::expr::{ExprImpl, ExprType, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ @@ -119,6 +120,20 @@ impl PredicatePushdown for LogicalIcebergScan { predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { + fn rw_literal_to_iceberg_datum(literal: &Literal) -> Option { + let Some(scalar) = literal.get_data() else { + return None; + }; + match scalar { + ScalarImpl::Bool(b) => Some(IcebergDatum::bool(*b)), + ScalarImpl::Int32(i) => Some(IcebergDatum::int(*i)), + ScalarImpl::Int64(i) => Some(IcebergDatum::long(*i)), + ScalarImpl::Float32(f) => Some(IcebergDatum::float(*f)), + ScalarImpl::Float64(f) => Some(IcebergDatum::double(*f)), + _ => None, + } + } + fn rw_expr_to_iceberg_predicate( expr: &ExprImpl, fields: &[Field], @@ -152,11 +167,12 @@ impl PredicatePushdown for LogicalIcebergScan { Some(IcebergPredicate::or(arg0, arg1)) } ExprType::Equal => match [&args[0], &args[1]] { - [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - todo!() - } - [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - todo!() + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] + | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.equal_to(datum)) } _ => None, }, From 425f80127acceb3641637d3bb5523178fb0449c0 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sat, 2 Nov 2024 09:05:41 +0800 Subject: [PATCH 05/42] convert rw expressions into iceberg predicates --- .../plan_node/logical_iceberg_scan.rs | 130 ++++++++++++++---- 1 file changed, 106 insertions(+), 24 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index f789994fa887f..0e77df99b7e59 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -176,30 +176,112 @@ impl PredicatePushdown for LogicalIcebergScan { } _ => None, }, - ExprType::NotEqual => { - todo!() - } - ExprType::GreaterThan => { - todo!() - } - ExprType::GreaterThanOrEqual => { - todo!() - } - ExprType::LessThan => { - todo!() - } - ExprType::LessThanOrEqual => { - todo!() - } - ExprType::IsNull => { - todo!() - } - ExprType::IsNotNull => { - todo!() - } - ExprType::In => { - todo!() - } + ExprType::NotEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] + | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.not_equal_to(datum)) + } + _ => None, + }, + ExprType::GreaterThan => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) + } + _ => None, + }, + ExprType::GreaterThanOrEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) + } + _ => None, + }, + ExprType::LessThan => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) + } + _ => None, + }, + ExprType::LessThanOrEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) + } + _ => None, + }, + ExprType::IsNull => match &args[0] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_null()) + } + _ => None, + }, + ExprType::IsNotNull => match &args[1] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_not_null()) + } + _ => None, + }, + ExprType::In => match &args[1] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let mut datums = Vec::with_capacity(args.len() - 1); + for arg in &args[1..] { + if let ExprImpl::Literal(l) = arg { + if let Some(datum) = rw_literal_to_iceberg_datum(l) { + datums.push(datum); + } else { + return None; + } + } else { + return None; + } + } + Some(reference.is_in(datums)) + } + _ => None, + }, _ => None, } } From bc197f40c0e646665393654d8f0cd431202d46fc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sat, 2 Nov 2024 17:03:49 +0800 Subject: [PATCH 06/42] add support for more literals --- .../plan_node/logical_iceberg_scan.rs | 27 ++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 0e77df99b7e59..091f46789217e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -14,11 +14,12 @@ use std::rc::Rc; +use chrono::Datelike; use iceberg::expr::{Predicate as IcebergPredicate, Reference}; use iceberg::spec::Datum as IcebergDatum; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::types::ScalarImpl; +use risingwave_common::types::{Decimal, ScalarImpl}; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; @@ -130,6 +131,30 @@ impl PredicatePushdown for LogicalIcebergScan { ScalarImpl::Int64(i) => Some(IcebergDatum::long(*i)), ScalarImpl::Float32(f) => Some(IcebergDatum::float(*f)), ScalarImpl::Float64(f) => Some(IcebergDatum::double(*f)), + ScalarImpl::Decimal(d) => { + let Decimal::Normalized(d) = d else { + return None; + }; + let Ok(d) = IcebergDatum::decimal(*d) else { + return None; + }; + Some(d) + } + ScalarImpl::Date(d) => { + let Ok(datum) = IcebergDatum::date_from_ymd(d.0.year(), d.0.month(), d.0.day()) + else { + return None; + }; + Some(datum) + } + ScalarImpl::Timestamp(t) => Some(IcebergDatum::timestamp_nanos( + t.0.and_utc().timestamp_nanos_opt()?, + )), + ScalarImpl::Timestamptz(t) => { + Some(IcebergDatum::timestamptz_micros(t.timestamp_micros())) + } + ScalarImpl::Utf8(s) => Some(IcebergDatum::string(s)), + ScalarImpl::Bytea(b) => Some(IcebergDatum::binary(b.clone())), _ => None, } } From f08446460e373cc213492fbf26a13efd02766291 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 4 Nov 2024 10:14:44 +0800 Subject: [PATCH 07/42] add predicate proto --- proto/batch_plan.proto | 100 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 100 insertions(+) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index b46230b2438d6..f871455a4aac4 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -66,6 +66,106 @@ message SourceNode { map secret_refs = 6; } +//enum IcebergPredicateType { +// ICEBERG_PREDICATE_UNSPECIFIED = 0; +// ICEBERG_PREDICATE_TRUE = 1; +// ICEBERG_PREDICATE_FALSE = 2; +// ICEBERG_PREDICATE_EQ = 3; +// ICEBERG_PREDICATE_NE = 4; +// ICEBERG_PREDICATE_LT = 5; +// ICEBERG_PREDICATE_LE = 6; +// ICEBERG_PREDICATE_GT = 7; +// ICEBERG_PREDICATE_GE = 8; +// ICEBERG_PREDICATE_IS_NULL = 9; +// ICEBERG_PREDICATE_IS_NOT_NULL = 10; +// ICEBERG_PREDICATE_IN = 11; +// ICEBERG_PREDICATE_NOT_IN = 12; +// ICEBERG_PREDICATE_AND = 13; +// ICEBERG_PREDICATE_OR = 14; +// ICEBERG_PREDICATE_NOT = 15; +//} + +message IcebergBinaryPredicate { + enum IcebergBinaryExprType { + ICEBERG_BINARY_EXPR_TYPE_UNSPECIFIED = 0; + ICEBERG_BINARY_EXPR_TYPE_AND = 1; + ICEBERG_BINARY_EXPR_TYPE_OR = 2; + } + IcebergBinaryExprType expr_type = 1; + IcebergPredicate left = 2; + IcebergPredicate right = 3; +} + +message IcebergUnaryPredicate { + enum IcebergUnaryExprType { + ICEBERG_UNARY_EXPR_TYPE_UNSPECIFIED = 0; + ICEBERG_UNARY_EXPR_TYPE_NOT = 1; + } + IcebergUnaryExprType expr_type = 1; + IcebergPredicate arg = 2; +} + +message IcebergDatum { + data.DataType datatype = 1; + data.Datum value = 2; +} + +message IcebergRefAndValuePredicate { + enum IcebergRefAndValueType { + ICEBERG_REF_AND_VALUE_TYPE_UNSPECIFIED = 0; + ICEBERG_REF_AND_VALUE_TYPE_EQ = 1; + ICEBERG_REF_AND_VALUE_TYPE_NE = 2; + ICEBERG_REF_AND_VALUE_TYPE_LT = 3; + ICEBERG_REF_AND_VALUE_TYPE_LE = 4; + ICEBERG_REF_AND_VALUE_TYPE_GT = 5; + ICEBERG_REF_AND_VALUE_TYPE_GE = 6; + ICEBERG_REF_AND_VALUE_TYPE_IS_NULL = 7; + ICEBERG_REF_AND_VALUE_TYPE_IS_NOT_NULL = 8; + ICEBERG_REF_AND_VALUE_TYPE_IN = 9; + } + IcebergRefAndValueType expr_type = 1; + string reference = 2; + IcebergDatum value = 3; +} + +message IcebergRefPredicate { + enum IcebergRefType { + ICEBERG_REF_TYPE_UNSPECIFIED = 0; + ICEBERG_REF_TYPE_EQ = 1; + ICEBERG_REF_TYPE_NE = 2; + ICEBERG_REF_TYPE_LT = 3; + ICEBERG_REF_TYPE_LE = 4; + ICEBERG_REF_TYPE_GT = 5; + ICEBERG_REF_TYPE_GE = 6; + ICEBERG_REF_TYPE_IS_NULL = 7; + ICEBERG_REF_TYPE_IS_NOT_NULL = 8; + ICEBERG_REF_TYPE_IN = 9; + } + IcebergRefType expr_type = 1; + string reference = 2; +} + +message IcebergSetPredicate { + enum IcebergSetType { + ICEBERG_SET_TYPE_UNSPECIFIED = 0; + ICEBERG_SET_TYPE_IN = 1; + } + IcebergSetType expr_type = 1; + string reference = 2; + repeated IcebergDatum values = 3; +} + +message IcebergPredicate { + oneof predicate_expr { + bool boolean = 1; + IcebergBinaryPredicate binary_predicate = 3; + IcebergUnaryPredicate unary_predicate = 4; + IcebergRefPredicate ref_predicate = 5; + IcebergRefAndValuePredicate ref_and_value_predicate = 6; + IcebergSetPredicate set_predicate = 7; + } +} + message IcebergScanNode { repeated plan_common.ColumnCatalog columns = 1; map with_properties = 2; From 2400e4aaaa880ed5d3739184fb75b93c3ca6867c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 4 Nov 2024 13:39:47 +0800 Subject: [PATCH 08/42] interim commit: use iceberg proto --- .../plan_node/logical_iceberg_scan.rs | 354 ++++++++++++------ 1 file changed, 244 insertions(+), 110 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 091f46789217e..214cf744987f7 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -15,11 +15,20 @@ use std::rc::Rc; use chrono::Datelike; -use iceberg::expr::{Predicate as IcebergPredicate, Reference}; -use iceberg::spec::Datum as IcebergDatum; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::types::{Decimal, ScalarImpl}; +use risingwave_common::types::{DataType, Decimal, ScalarImpl}; +use risingwave_common::util::value_encoding::DatumToProtoExt; +use risingwave_pb::batch_plan::iceberg_binary_predicate::IcebergBinaryExprType; +use risingwave_pb::batch_plan::iceberg_predicate::PredicateExpr; +use risingwave_pb::batch_plan::iceberg_ref_and_value_predicate::IcebergRefAndValueType; +use risingwave_pb::batch_plan::iceberg_ref_predicate::IcebergRefType; +use risingwave_pb::batch_plan::iceberg_unary_predicate::IcebergUnaryExprType; +use risingwave_pb::batch_plan::{ + IcebergBinaryExprType, IcebergBinaryPredicate, IcebergDatum, IcebergPredicate, + IcebergRefAndValuePredicate, IcebergRefAndValueType, IcebergSetPredicate, IcebergUnaryExprType, + IcebergUnaryPredicate, +}; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; @@ -29,7 +38,7 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; -use crate::expr::{ExprImpl, ExprType, Literal}; +use crate::expr::{ExprImpl, ExprType, InputRef, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ @@ -110,7 +119,7 @@ impl ExprVisitable for LogicalIcebergScan {} impl PredicatePushdown for LogicalIcebergScan { /// NOTE(kwannoel): /// 1. We expect it to be constant folded - /// 2. We don't convert `inputRefs` of type boolean directly to IcebergPredicates. + /// 2. We don't convert `inputRefs` of type boolean directly to `IcebergPredicates`. /// 3. The leaf nodes are always logical comparison operators: /// `Equal`, `NotEqual`, `GreaterThan`, /// `GreaterThanOrEqual`, `LessThan`, `LessThanOrEqual`. @@ -122,39 +131,60 @@ impl PredicatePushdown for LogicalIcebergScan { _ctx: &mut PredicatePushdownContext, ) -> PlanRef { fn rw_literal_to_iceberg_datum(literal: &Literal) -> Option { - let Some(scalar) = literal.get_data() else { + let datum @ Some(scalar) = literal.get_data() else { return None; }; + match scalar { - ScalarImpl::Bool(b) => Some(IcebergDatum::bool(*b)), - ScalarImpl::Int32(i) => Some(IcebergDatum::int(*i)), - ScalarImpl::Int64(i) => Some(IcebergDatum::long(*i)), - ScalarImpl::Float32(f) => Some(IcebergDatum::float(*f)), - ScalarImpl::Float64(f) => Some(IcebergDatum::double(*f)), + ScalarImpl::Bool(b) => Some(IcebergDatum { + datatype: DataType::Boolean.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Int32(i) => Some(IcebergDatum { + datatype: DataType::Int32.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Int64(i) => Some(IcebergDatum { + datatype: DataType::Int64.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Float32(f) => Some(IcebergDatum { + datatype: DataType::Float32.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Float64(f) => Some(IcebergDatum { + datatype: DataType::Float64.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), ScalarImpl::Decimal(d) => { let Decimal::Normalized(d) = d else { return None; }; - let Ok(d) = IcebergDatum::decimal(*d) else { - return None; - }; - Some(d) - } - ScalarImpl::Date(d) => { - let Ok(datum) = IcebergDatum::date_from_ymd(d.0.year(), d.0.month(), d.0.day()) - else { - return None; - }; - Some(datum) + Some(IcebergDatum { + datatype: DataType::Decimal.to_protobuf().into(), + value: datum.to_protobuf().into(), + }) } - ScalarImpl::Timestamp(t) => Some(IcebergDatum::timestamp_nanos( - t.0.and_utc().timestamp_nanos_opt()?, - )), - ScalarImpl::Timestamptz(t) => { - Some(IcebergDatum::timestamptz_micros(t.timestamp_micros())) - } - ScalarImpl::Utf8(s) => Some(IcebergDatum::string(s)), - ScalarImpl::Bytea(b) => Some(IcebergDatum::binary(b.clone())), + ScalarImpl::Date(d) => Some(IcebergDatum { + datatype: DataType::Date.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Timestamp(t) => Some(IcebergDatum { + datatype: DataType::Timestamp.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Timestamptz(t) => Some(IcebergDatum { + datatype: DataType::Timestamptz.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Utf8(s) => Some(IcebergDatum { + datatype: DataType::Varchar.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), + ScalarImpl::Bytea(b) => Some(IcebergDatum { + datatype: DataType::Bytea.to_protobuf().into(), + value: datum.to_protobuf().into(), + }), _ => None, } } @@ -164,133 +194,210 @@ impl PredicatePushdown for LogicalIcebergScan { fields: &[Field], ) -> Option { match expr { - ExprImpl::Literal(l) => match l.get_data() { - Some(ScalarImpl::Bool(b)) => { - if *b { - Some(IcebergPredicate::AlwaysTrue) - } else { - Some(IcebergPredicate::AlwaysFalse) - } + ExprImpl::Literal(l) => { + if let Some(ScalarImpl::Bool(b)) = l.get_data() { + Some(IcebergPredicate { + predicate_expr: Some(PredicateExpr::Boolean(*b)), + }) + } else { + None } - _ => None, - }, + } ExprImpl::FunctionCall(f) => { + fn create_binary_predicate( + expr_type: IcebergBinaryExprType, + left: &ExprImpl, + right: &ExprImpl, + fields: &[Field], + ) -> Option { + let left = rw_expr_to_iceberg_predicate(left, fields)?; + let right = rw_expr_to_iceberg_predicate(right, fields)?; + Some(IcebergPredicate { + predicate_expr: Some(PredicateExpr::BinaryPredicate(Box::new( + IcebergBinaryPredicate { + expr_type: expr_type as i32, + left: Some(Box::new(left)), + right: Some(Box::new(right)), + }, + ))), + }) + } + + fn create_ref_predicate( + expr_type: IcebergRefType, + reference: &InputRef, + fields: &[Field], + ) -> Option { + let reference = fields[reference.index].name.clone(); + Some(IcebergPredicate { + predicate_expr: Some(PredicateExpr::RefPredicate( + IcebergRefPredicate { + expr_type: expr_type as i32, + reference, + }, + )), + }) + } + + fn create_ref_and_value_predicate( + expr_type: IcebergRefAndValueType, + reference: &InputRef, + arg: &Literal, + fields: &[Field], + ) -> Option { + let reference = fields[reference.index].name.clone(); + let arg = rw_literal_to_iceberg_datum(arg)?; + Some(IcebergPredicate { + predicate_expr: Some(PredicateExpr::RefAndValuePredicate( + IcebergRefAndValuePredicate { + expr_type: expr_type as i32, + reference, + value: Some(arg), + }, + )), + }) + } + let args = f.inputs(); match f.func_type() { ExprType::Not => { let arg = rw_expr_to_iceberg_predicate(&args[0], fields)?; - Some(IcebergPredicate::negate(arg)) - } - ExprType::And => { - let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; - let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; - Some(IcebergPredicate::and(arg0, arg1)) - } - ExprType::Or => { - let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; - let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; - Some(IcebergPredicate::or(arg0, arg1)) + Some(IcebergPredicate { + predicate_expr: Some(PredicateExpr::UnaryPredicate(Box::new( + IcebergUnaryPredicate { + expr_type: IcebergUnaryExprType::Not as i32, + arg: Some(Box::new(arg)), + }, + ))), + }) } + ExprType::And => create_binary_predicate( + IcebergBinaryExprType::And, + &args[0], + &args[1], + fields, + ), + ExprType::Or => create_binary_predicate( + IcebergBinaryExprType::Or, + &args[0], + &args[1], + fields, + ), ExprType::Equal => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.equal_to(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Eq, + lhs, + rhs, + fields, + ) } _ => None, }, ExprType::NotEqual => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.not_equal_to(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Ne, + lhs, + rhs, + fields, + ) } _ => None, }, ExprType::GreaterThan => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Gt, + lhs, + rhs, + fields, + ) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than_or_equal_to(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Le, + lhs, + rhs, + fields, + ) } _ => None, }, ExprType::GreaterThanOrEqual => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than_or_equal_to(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Ge, + lhs, + rhs, + fields, + ) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Lt, + lhs, + rhs, + fields, + ) } _ => None, }, ExprType::LessThan => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Lt, + lhs, + rhs, + fields, + ) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than_or_equal_to(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Ge, + lhs, + rhs, + fields, + ) } _ => None, }, ExprType::LessThanOrEqual => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than_or_equal_to(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Le, + lhs, + rhs, + fields, + ) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than(datum)) + create_ref_and_value_predicate( + IcebergRefAndValueType::Gt, + lhs, + rhs, + fields, + ) } _ => None, }, ExprType::IsNull => match &args[0] { - ExprImpl::InputRef(lhs) => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - Some(reference.is_null()) + ExprImpl::InputRef(reference) => { + create_ref_predicate(IcebergRefType::IsNull, reference, fields) } _ => None, }, - ExprType::IsNotNull => match &args[1] { - ExprImpl::InputRef(lhs) => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - Some(reference.is_not_null()) + ExprType::IsNotNull => match &args[0] { + ExprImpl::InputRef(reference) => { + create_ref_predicate(IcebergRefType::IsNotNull, reference, fields) } _ => None, }, - ExprType::In => match &args[1] { + ExprType::In => match &args[0] { ExprImpl::InputRef(lhs) => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); + let reference = fields[lhs.index].name.clone(); let mut datums = Vec::with_capacity(args.len() - 1); for arg in &args[1..] { if let ExprImpl::Literal(l) = arg { @@ -303,7 +410,15 @@ impl PredicatePushdown for LogicalIcebergScan { return None; } } - Some(reference.is_in(datums)) + Some(IcebergPredicate { + predicate_expr: Some(PredicateExpr::SetPredicate( + IcebergSetPredicate { + expr_type: IcebergRefAndValueType::In as i32, + reference, + values: datums, + }, + )), + }) } _ => None, }, @@ -316,21 +431,40 @@ impl PredicatePushdown for LogicalIcebergScan { fn rw_predicate_to_iceberg_predicate( predicate: Condition, fields: &[Field], - ) -> IcebergPredicate { + ) -> (Option<(IcebergPredicate, Condition)>) { if predicate.always_true() { - return IcebergPredicate::AlwaysTrue; + return None; } let mut conjunctions = predicate.conjunctions; let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); - let rw_condition_root = conjunctions.pop().unwrap(); - let iceberg_condition_root = rw_expr_to_iceberg_predicate(&rw_condition_root, fields); + let mut iceberg_condition_root; + loop { + let Some(rw_condition_root) = conjunctions.pop() else { + return None; + }; + match rw_expr_to_iceberg_predicate(&rw_condition_root, fields) { + Some(iceberg_predicate) => { + iceberg_condition_root = iceberg_predicate; + break; + } + None => ignored_conjunctions.push(rw_condition_root), + } + } for rw_condition in conjunctions { match rw_expr_to_iceberg_predicate(&rw_condition, fields) { - Some(iceberg_predicate) => ignored_conjunctions.push(rw_condition), - None => {} + Some(iceberg_predicate) => { + iceberg_condition_root = + IcebergPredicate::and(iceberg_condition_root, iceberg_predicate); + } + None => ignored_conjunctions.push(rw_condition), } } - todo!() + Some(( + iceberg_condition_root, + Condition { + conjunctions: ignored_conjunctions, + }, + )) } let schema = self.schema(); From a28e79bf01f8dbd79aef3b10a548034f6245ce14 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 4 Nov 2024 14:26:02 +0800 Subject: [PATCH 09/42] change predicate_pushdown return --- .../plan_node/logical_iceberg_scan.rs | 51 ++++++++++--------- 1 file changed, 28 insertions(+), 23 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 214cf744987f7..8d6c99e432f5d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -14,9 +14,8 @@ use std::rc::Rc; -use chrono::Datelike; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::Field; use risingwave_common::types::{DataType, Decimal, ScalarImpl}; use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_pb::batch_plan::iceberg_binary_predicate::IcebergBinaryExprType; @@ -25,9 +24,8 @@ use risingwave_pb::batch_plan::iceberg_ref_and_value_predicate::IcebergRefAndVal use risingwave_pb::batch_plan::iceberg_ref_predicate::IcebergRefType; use risingwave_pb::batch_plan::iceberg_unary_predicate::IcebergUnaryExprType; use risingwave_pb::batch_plan::{ - IcebergBinaryExprType, IcebergBinaryPredicate, IcebergDatum, IcebergPredicate, - IcebergRefAndValuePredicate, IcebergRefAndValueType, IcebergSetPredicate, IcebergUnaryExprType, - IcebergUnaryPredicate, + IcebergBinaryPredicate, IcebergDatum, IcebergPredicate, IcebergRefAndValuePredicate, + IcebergRefPredicate, IcebergSetPredicate, IcebergUnaryPredicate, }; use super::generic::GenericPlanRef; @@ -136,23 +134,23 @@ impl PredicatePushdown for LogicalIcebergScan { }; match scalar { - ScalarImpl::Bool(b) => Some(IcebergDatum { + ScalarImpl::Bool(_) => Some(IcebergDatum { datatype: DataType::Boolean.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Int32(i) => Some(IcebergDatum { + ScalarImpl::Int32(_) => Some(IcebergDatum { datatype: DataType::Int32.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Int64(i) => Some(IcebergDatum { + ScalarImpl::Int64(_) => Some(IcebergDatum { datatype: DataType::Int64.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Float32(f) => Some(IcebergDatum { + ScalarImpl::Float32(_) => Some(IcebergDatum { datatype: DataType::Float32.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Float64(f) => Some(IcebergDatum { + ScalarImpl::Float64(_) => Some(IcebergDatum { datatype: DataType::Float64.to_protobuf().into(), value: datum.to_protobuf().into(), }), @@ -165,23 +163,23 @@ impl PredicatePushdown for LogicalIcebergScan { value: datum.to_protobuf().into(), }) } - ScalarImpl::Date(d) => Some(IcebergDatum { + ScalarImpl::Date(_) => Some(IcebergDatum { datatype: DataType::Date.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Timestamp(t) => Some(IcebergDatum { + ScalarImpl::Timestamp(_) => Some(IcebergDatum { datatype: DataType::Timestamp.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Timestamptz(t) => Some(IcebergDatum { + ScalarImpl::Timestamptz(_) => Some(IcebergDatum { datatype: DataType::Timestamptz.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Utf8(s) => Some(IcebergDatum { + ScalarImpl::Utf8(_) => Some(IcebergDatum { datatype: DataType::Varchar.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Bytea(b) => Some(IcebergDatum { + ScalarImpl::Bytea(_) => Some(IcebergDatum { datatype: DataType::Bytea.to_protobuf().into(), value: datum.to_protobuf().into(), }), @@ -431,16 +429,16 @@ impl PredicatePushdown for LogicalIcebergScan { fn rw_predicate_to_iceberg_predicate( predicate: Condition, fields: &[Field], - ) -> (Option<(IcebergPredicate, Condition)>) { + ) -> (Condition, Option) { if predicate.always_true() { - return None; + return (predicate, None); } let mut conjunctions = predicate.conjunctions; let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); let mut iceberg_condition_root; loop { let Some(rw_condition_root) = conjunctions.pop() else { - return None; + return (Condition { conjunctions }, None); }; match rw_expr_to_iceberg_predicate(&rw_condition_root, fields) { Some(iceberg_predicate) => { @@ -453,18 +451,25 @@ impl PredicatePushdown for LogicalIcebergScan { for rw_condition in conjunctions { match rw_expr_to_iceberg_predicate(&rw_condition, fields) { Some(iceberg_predicate) => { - iceberg_condition_root = - IcebergPredicate::and(iceberg_condition_root, iceberg_predicate); + iceberg_condition_root = IcebergPredicate { + predicate_expr: Some(PredicateExpr::BinaryPredicate(Box::new( + IcebergBinaryPredicate { + expr_type: IcebergBinaryExprType::And as i32, + left: Some(Box::new(iceberg_condition_root)), + right: Some(Box::new(iceberg_predicate)), + }, + ))), + } } None => ignored_conjunctions.push(rw_condition), } } - Some(( - iceberg_condition_root, + ( Condition { conjunctions: ignored_conjunctions, }, - )) + Some(iceberg_condition_root), + ) } let schema = self.schema(); From 43ba97d0dda2950c7f957343388c9397c46ddc77 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 4 Nov 2024 16:49:03 +0800 Subject: [PATCH 10/42] derive eq, hash for iceberg predicate --- proto/batch_plan.proto | 1 + .../plan_node/logical_iceberg_scan.rs | 51 +++++++++++++++++-- src/prost/build.rs | 14 +++++ 3 files changed, 62 insertions(+), 4 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index f871455a4aac4..44fad3ad30ef9 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -145,6 +145,7 @@ message IcebergRefPredicate { string reference = 2; } +/// a in (...) message IcebergSetPredicate { enum IcebergSetType { ICEBERG_SET_TYPE_UNSPECIFIED = 0; diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 8d6c99e432f5d..c526fa07381eb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -50,6 +50,7 @@ use crate::utils::{ColIndexMapping, Condition}; pub struct LogicalIcebergScan { pub base: PlanBase, pub core: generic::Source, + pub iceberg_predicate: IcebergPredicate, } impl LogicalIcebergScan { @@ -61,7 +62,13 @@ impl LogicalIcebergScan { assert!(logical_source.output_exprs.is_none()); - LogicalIcebergScan { base, core } + LogicalIcebergScan { + base, + core, + iceberg_predicate: IcebergPredicate { + predicate_expr: Some(PredicateExpr::Boolean(true)), + }, + } } pub fn source_catalog(&self) -> Option> { @@ -76,8 +83,32 @@ impl LogicalIcebergScan { .map(|idx| core.column_catalog[*idx].clone()) .collect(); let base = PlanBase::new_logical_with_core(&core); + let iceberg_predicate = self.iceberg_predicate.clone(); - LogicalIcebergScan { base, core } + LogicalIcebergScan { + base, + core, + iceberg_predicate, + } + } + + fn clone_with_iceberg_predicate(&self, iceberg_predicate: IcebergPredicate) -> Self { + let base = self.base.clone(); + let core = self.core.clone(); + let iceberg_predicate = IcebergPredicate { + predicate_expr: Some(PredicateExpr::BinaryPredicate(Box::new( + IcebergBinaryPredicate { + expr_type: IcebergBinaryExprType::And as i32, + left: Some(Box::new(self.iceberg_predicate.clone())), + right: Some(Box::new(iceberg_predicate)), + }, + ))), + }; + LogicalIcebergScan { + base, + core, + iceberg_predicate, + } } } @@ -155,7 +186,7 @@ impl PredicatePushdown for LogicalIcebergScan { value: datum.to_protobuf().into(), }), ScalarImpl::Decimal(d) => { - let Decimal::Normalized(d) = d else { + let Decimal::Normalized(_) = d else { return None; }; Some(IcebergDatum { @@ -237,6 +268,10 @@ impl PredicatePushdown for LogicalIcebergScan { }) } + /// predicate: a < 1 + /// + /// a: reference + /// 1: value. fn create_ref_and_value_predicate( expr_type: IcebergRefAndValueType, reference: &InputRef, @@ -474,8 +509,16 @@ impl PredicatePushdown for LogicalIcebergScan { let schema = self.schema(); let fields = &schema.fields; + + let (rw_predicate, iceberg_predicate_opt) = + rw_predicate_to_iceberg_predicate(predicate, fields); // No pushdown. - LogicalFilter::create(self.clone().into(), predicate) + let this = if let Some(iceberg_predicate) = iceberg_predicate_opt { + self.clone_with_iceberg_predicate(iceberg_predicate).into() + } else { + self.clone().into() + }; + LogicalFilter::create(this, rw_predicate) } } diff --git a/src/prost/build.rs b/src/prost/build.rs index c4744e14c1b60..c42856054d289 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -127,6 +127,20 @@ fn main() -> Result<(), Box> { .type_attribute("data.Datum", "#[derive(Eq, Hash)]") .type_attribute("expr.FunctionCall", "#[derive(Eq, Hash)]") .type_attribute("expr.UserDefinedFunction", "#[derive(Eq, Hash)]") + .type_attribute("batch_plan.IcebergBinaryPredicate", "#[derive(Eq, Hash)]") + .type_attribute("batch_plan.IcebergUnaryPredicate", "#[derive(Eq, Hash)]") + .type_attribute("batch_plan.IcebergDatum", "#[derive(Eq, Hash)]") + .type_attribute( + "batch_plan.IcebergRefAndValuePredicate", + "#[derive(Eq, Hash)]", + ) + .type_attribute("batch_plan.IcebergRefPredicate", "#[derive(Eq, Hash)]") + .type_attribute("batch_plan.IcebergSetPredicate", "#[derive(Eq, Hash)]") + .type_attribute( + "batch_plan.IcebergPredicate.predicate_expr", + "#[derive(Eq, Hash)]", + ) + .type_attribute("batch_plan.IcebergPredicate", "#[derive(Eq, Hash)]") .type_attribute( "plan_common.ColumnDesc.generated_or_default_column", "#[derive(Eq, Hash)]", From 09268d646223d373e6333ea62b9e100490ea74b7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 4 Nov 2024 17:01:28 +0800 Subject: [PATCH 11/42] interim commit: add iceberg_predicate to batch --- proto/batch_plan.proto | 1 + .../src/optimizer/plan_node/batch_iceberg_scan.rs | 13 ++++++++++--- .../src/optimizer/plan_node/logical_iceberg_scan.rs | 3 ++- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 44fad3ad30ef9..67346112987ca 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -172,6 +172,7 @@ message IcebergScanNode { map with_properties = 2; repeated bytes split = 3; map secret_refs = 4; + IcebergPredicate predicate = 5; } message FileScanNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs index 4333fcaa3e90a..3a1f753e78c18 100644 --- a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs @@ -16,7 +16,7 @@ use std::rc::Rc; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; -use risingwave_pb::batch_plan::IcebergScanNode; +use risingwave_pb::batch_plan::{IcebergPredicate, IcebergScanNode}; use risingwave_sqlparser::ast::AsOf; use super::batch::prelude::*; @@ -33,10 +33,11 @@ use crate::optimizer::property::{Distribution, Order}; pub struct BatchIcebergScan { pub base: PlanBase, pub core: generic::Source, + pub iceberg_predicate: IcebergPredicate, } impl BatchIcebergScan { - pub fn new(core: generic::Source) -> Self { + pub fn new(core: generic::Source, iceberg_predicate: IcebergPredicate) -> Self { let base = PlanBase::new_batch_with_core( &core, // Use `Single` by default, will be updated later with `clone_with_dist`. @@ -44,7 +45,11 @@ impl BatchIcebergScan { Order::any(), ); - Self { base, core } + Self { + base, + core, + iceberg_predicate, + } } pub fn column_names(&self) -> Vec<&str> { @@ -62,6 +67,7 @@ impl BatchIcebergScan { Self { base, core: self.core.clone(), + iceberg_predicate: self.iceberg_predicate.clone(), } } @@ -109,6 +115,7 @@ impl ToBatchPb for BatchIcebergScan { with_properties, split: vec![], secret_refs, + predicate: Some(self.iceberg_predicate.clone()), }) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index c526fa07381eb..54622deb1a491 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -524,7 +524,8 @@ impl PredicatePushdown for LogicalIcebergScan { impl ToBatch for LogicalIcebergScan { fn to_batch(&self) -> Result { - let plan: PlanRef = BatchIcebergScan::new(self.core.clone()).into(); + let plan: PlanRef = + BatchIcebergScan::new(self.core.clone(), self.iceberg_predicate.clone()).into(); Ok(plan) } } From 6b7cf2df2593626725749528f6facc93ff6e3529 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 5 Nov 2024 21:42:21 +0800 Subject: [PATCH 12/42] add fetch_parameters --- src/frontend/src/scheduler/plan_fragmenter.rs | 43 ++++++++++++++----- 1 file changed, 32 insertions(+), 11 deletions(-) diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 90984750bc460..4db8302bb407c 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -22,6 +22,7 @@ use anyhow::anyhow; use async_recursion::async_recursion; use enum_as_inner::EnumAsInner; use futures::TryStreamExt; +use iceberg::expr::Predicate as IcebergPredicate; use itertools::Itertools; use pgwire::pg_server::SessionId; use risingwave_batch::error::BatchError; @@ -268,11 +269,25 @@ impl Query { } } +#[derive(Debug, Clone)] +enum SourceFetchParameters { + IcebergPredicate(IcebergPredicate), + KafkaTimebound { + lower: Option, + upper: Option, + }, + Empty, +} + #[derive(Debug, Clone)] pub struct SourceFetchInfo { pub schema: Schema, + /// These are user-configured connector properties. + /// e.g. host, username, etc... pub connector: ConnectorProperties, - pub timebound: (Option, Option), + /// These parameters are internally derived by the plan node. + /// e.g. predicate pushdown for iceberg, timebound for kafka. + pub fetch_parameters: SourceFetchParameters, pub as_of: Option, } @@ -295,13 +310,16 @@ impl SourceScanInfo { unreachable!("Never call complete when SourceScanInfo is already complete") } }; - match fetch_info.connector { - ConnectorProperties::Kafka(prop) => { + match (fetch_info.connector, fetch_info.fetch_parameters) { + ( + ConnectorProperties::Kafka(prop), + SourceFetchParameters::KafkaTimebound { lower, upper }, + ) => { let mut kafka_enumerator = KafkaSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) .await?; let split_info = kafka_enumerator - .list_splits_batch(fetch_info.timebound.0, fetch_info.timebound.1) + .list_splits_batch(lower, upper) .await? .into_iter() .map(SplitImpl::Kafka) @@ -309,7 +327,7 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(split_info)) } - ConnectorProperties::OpendalS3(prop) => { + (ConnectorProperties::OpendalS3(prop), SourceFetchParameters::Empty) => { let lister: OpendalEnumerator = OpendalEnumerator::new_s3_source(prop.s3_properties, prop.assume_role)?; let stream = build_opendal_fs_list_for_batch(lister); @@ -322,7 +340,7 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } - ConnectorProperties::Gcs(prop) => { + (ConnectorProperties::Gcs(prop), SourceFetchParameters::Empty) => { let lister: OpendalEnumerator = OpendalEnumerator::new_gcs_source(*prop)?; let stream = build_opendal_fs_list_for_batch(lister); @@ -331,7 +349,7 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } - ConnectorProperties::Azblob(prop) => { + (ConnectorProperties::Azblob(prop), SourceFetchParameters::Empty) => { let lister: OpendalEnumerator = OpendalEnumerator::new_azblob_source(*prop)?; let stream = build_opendal_fs_list_for_batch(lister); @@ -340,7 +358,7 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } - ConnectorProperties::Iceberg(prop) => { + (ConnectorProperties::Iceberg(prop), SourceFetchParameters::IcebergPredicate(_)) => { let iceberg_enumerator = IcebergSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) .await?; @@ -1068,7 +1086,10 @@ impl BatchPlanFragmenter { return Ok(Some(SourceScanInfo::new(SourceFetchInfo { schema: batch_kafka_scan.base.schema().clone(), connector: property, - timebound: timestamp_bound, + fetch_parameters: SourceFetchParameters::KafkaTimebound { + lower: timestamp_bound.0, + upper: timestamp_bound.1, + }, as_of: None, }))); } @@ -1082,7 +1103,7 @@ impl BatchPlanFragmenter { return Ok(Some(SourceScanInfo::new(SourceFetchInfo { schema: batch_iceberg_scan.base.schema().clone(), connector: property, - timebound: (None, None), + fetch_parameters: SourceFetchParameters::Empty, as_of, }))); } @@ -1097,7 +1118,7 @@ impl BatchPlanFragmenter { return Ok(Some(SourceScanInfo::new(SourceFetchInfo { schema: source_node.base.schema().clone(), connector: property, - timebound: (None, None), + fetch_parameters: SourceFetchParameters::Empty, as_of, }))); } From e4154ecc01778e16cb0f5aa3dd918d6384b3885f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 08:45:38 +0800 Subject: [PATCH 13/42] Revert "derive eq, hash for iceberg predicate" This reverts commit 706b9d86147e446f7c9592eca5df0d3f140323ef. --- proto/batch_plan.proto | 1 - .../plan_node/logical_iceberg_scan.rs | 51 ++----------------- src/prost/build.rs | 14 ----- 3 files changed, 4 insertions(+), 62 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 67346112987ca..f3980f5e5a765 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -145,7 +145,6 @@ message IcebergRefPredicate { string reference = 2; } -/// a in (...) message IcebergSetPredicate { enum IcebergSetType { ICEBERG_SET_TYPE_UNSPECIFIED = 0; diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 54622deb1a491..d8d54a4efbf15 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -50,7 +50,6 @@ use crate::utils::{ColIndexMapping, Condition}; pub struct LogicalIcebergScan { pub base: PlanBase, pub core: generic::Source, - pub iceberg_predicate: IcebergPredicate, } impl LogicalIcebergScan { @@ -62,13 +61,7 @@ impl LogicalIcebergScan { assert!(logical_source.output_exprs.is_none()); - LogicalIcebergScan { - base, - core, - iceberg_predicate: IcebergPredicate { - predicate_expr: Some(PredicateExpr::Boolean(true)), - }, - } + LogicalIcebergScan { base, core } } pub fn source_catalog(&self) -> Option> { @@ -83,32 +76,8 @@ impl LogicalIcebergScan { .map(|idx| core.column_catalog[*idx].clone()) .collect(); let base = PlanBase::new_logical_with_core(&core); - let iceberg_predicate = self.iceberg_predicate.clone(); - LogicalIcebergScan { - base, - core, - iceberg_predicate, - } - } - - fn clone_with_iceberg_predicate(&self, iceberg_predicate: IcebergPredicate) -> Self { - let base = self.base.clone(); - let core = self.core.clone(); - let iceberg_predicate = IcebergPredicate { - predicate_expr: Some(PredicateExpr::BinaryPredicate(Box::new( - IcebergBinaryPredicate { - expr_type: IcebergBinaryExprType::And as i32, - left: Some(Box::new(self.iceberg_predicate.clone())), - right: Some(Box::new(iceberg_predicate)), - }, - ))), - }; - LogicalIcebergScan { - base, - core, - iceberg_predicate, - } + LogicalIcebergScan { base, core } } } @@ -186,7 +155,7 @@ impl PredicatePushdown for LogicalIcebergScan { value: datum.to_protobuf().into(), }), ScalarImpl::Decimal(d) => { - let Decimal::Normalized(_) = d else { + let Decimal::Normalized(d) = d else { return None; }; Some(IcebergDatum { @@ -268,10 +237,6 @@ impl PredicatePushdown for LogicalIcebergScan { }) } - /// predicate: a < 1 - /// - /// a: reference - /// 1: value. fn create_ref_and_value_predicate( expr_type: IcebergRefAndValueType, reference: &InputRef, @@ -509,16 +474,8 @@ impl PredicatePushdown for LogicalIcebergScan { let schema = self.schema(); let fields = &schema.fields; - - let (rw_predicate, iceberg_predicate_opt) = - rw_predicate_to_iceberg_predicate(predicate, fields); // No pushdown. - let this = if let Some(iceberg_predicate) = iceberg_predicate_opt { - self.clone_with_iceberg_predicate(iceberg_predicate).into() - } else { - self.clone().into() - }; - LogicalFilter::create(this, rw_predicate) + LogicalFilter::create(self.clone().into(), predicate) } } diff --git a/src/prost/build.rs b/src/prost/build.rs index c42856054d289..c4744e14c1b60 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -127,20 +127,6 @@ fn main() -> Result<(), Box> { .type_attribute("data.Datum", "#[derive(Eq, Hash)]") .type_attribute("expr.FunctionCall", "#[derive(Eq, Hash)]") .type_attribute("expr.UserDefinedFunction", "#[derive(Eq, Hash)]") - .type_attribute("batch_plan.IcebergBinaryPredicate", "#[derive(Eq, Hash)]") - .type_attribute("batch_plan.IcebergUnaryPredicate", "#[derive(Eq, Hash)]") - .type_attribute("batch_plan.IcebergDatum", "#[derive(Eq, Hash)]") - .type_attribute( - "batch_plan.IcebergRefAndValuePredicate", - "#[derive(Eq, Hash)]", - ) - .type_attribute("batch_plan.IcebergRefPredicate", "#[derive(Eq, Hash)]") - .type_attribute("batch_plan.IcebergSetPredicate", "#[derive(Eq, Hash)]") - .type_attribute( - "batch_plan.IcebergPredicate.predicate_expr", - "#[derive(Eq, Hash)]", - ) - .type_attribute("batch_plan.IcebergPredicate", "#[derive(Eq, Hash)]") .type_attribute( "plan_common.ColumnDesc.generated_or_default_column", "#[derive(Eq, Hash)]", From a1be44d1bfcdd5e6ade24bb0b15029dd64a0f035 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 08:45:44 +0800 Subject: [PATCH 14/42] Revert "change predicate_pushdown return" This reverts commit d55dbc35eced7b2566fa6cb4cce070ae9b349912. --- .../plan_node/logical_iceberg_scan.rs | 51 +++++++++---------- 1 file changed, 23 insertions(+), 28 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index d8d54a4efbf15..9f2c05f6e7b91 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -14,8 +14,9 @@ use std::rc::Rc; +use chrono::Datelike; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::Field; +use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::{DataType, Decimal, ScalarImpl}; use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_pb::batch_plan::iceberg_binary_predicate::IcebergBinaryExprType; @@ -24,8 +25,9 @@ use risingwave_pb::batch_plan::iceberg_ref_and_value_predicate::IcebergRefAndVal use risingwave_pb::batch_plan::iceberg_ref_predicate::IcebergRefType; use risingwave_pb::batch_plan::iceberg_unary_predicate::IcebergUnaryExprType; use risingwave_pb::batch_plan::{ - IcebergBinaryPredicate, IcebergDatum, IcebergPredicate, IcebergRefAndValuePredicate, - IcebergRefPredicate, IcebergSetPredicate, IcebergUnaryPredicate, + IcebergBinaryExprType, IcebergBinaryPredicate, IcebergDatum, IcebergPredicate, + IcebergRefAndValuePredicate, IcebergRefAndValueType, IcebergSetPredicate, IcebergUnaryExprType, + IcebergUnaryPredicate, }; use super::generic::GenericPlanRef; @@ -134,23 +136,23 @@ impl PredicatePushdown for LogicalIcebergScan { }; match scalar { - ScalarImpl::Bool(_) => Some(IcebergDatum { + ScalarImpl::Bool(b) => Some(IcebergDatum { datatype: DataType::Boolean.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Int32(_) => Some(IcebergDatum { + ScalarImpl::Int32(i) => Some(IcebergDatum { datatype: DataType::Int32.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Int64(_) => Some(IcebergDatum { + ScalarImpl::Int64(i) => Some(IcebergDatum { datatype: DataType::Int64.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Float32(_) => Some(IcebergDatum { + ScalarImpl::Float32(f) => Some(IcebergDatum { datatype: DataType::Float32.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Float64(_) => Some(IcebergDatum { + ScalarImpl::Float64(f) => Some(IcebergDatum { datatype: DataType::Float64.to_protobuf().into(), value: datum.to_protobuf().into(), }), @@ -163,23 +165,23 @@ impl PredicatePushdown for LogicalIcebergScan { value: datum.to_protobuf().into(), }) } - ScalarImpl::Date(_) => Some(IcebergDatum { + ScalarImpl::Date(d) => Some(IcebergDatum { datatype: DataType::Date.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Timestamp(_) => Some(IcebergDatum { + ScalarImpl::Timestamp(t) => Some(IcebergDatum { datatype: DataType::Timestamp.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Timestamptz(_) => Some(IcebergDatum { + ScalarImpl::Timestamptz(t) => Some(IcebergDatum { datatype: DataType::Timestamptz.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Utf8(_) => Some(IcebergDatum { + ScalarImpl::Utf8(s) => Some(IcebergDatum { datatype: DataType::Varchar.to_protobuf().into(), value: datum.to_protobuf().into(), }), - ScalarImpl::Bytea(_) => Some(IcebergDatum { + ScalarImpl::Bytea(b) => Some(IcebergDatum { datatype: DataType::Bytea.to_protobuf().into(), value: datum.to_protobuf().into(), }), @@ -429,16 +431,16 @@ impl PredicatePushdown for LogicalIcebergScan { fn rw_predicate_to_iceberg_predicate( predicate: Condition, fields: &[Field], - ) -> (Condition, Option) { + ) -> (Option<(IcebergPredicate, Condition)>) { if predicate.always_true() { - return (predicate, None); + return None; } let mut conjunctions = predicate.conjunctions; let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); let mut iceberg_condition_root; loop { let Some(rw_condition_root) = conjunctions.pop() else { - return (Condition { conjunctions }, None); + return None; }; match rw_expr_to_iceberg_predicate(&rw_condition_root, fields) { Some(iceberg_predicate) => { @@ -451,25 +453,18 @@ impl PredicatePushdown for LogicalIcebergScan { for rw_condition in conjunctions { match rw_expr_to_iceberg_predicate(&rw_condition, fields) { Some(iceberg_predicate) => { - iceberg_condition_root = IcebergPredicate { - predicate_expr: Some(PredicateExpr::BinaryPredicate(Box::new( - IcebergBinaryPredicate { - expr_type: IcebergBinaryExprType::And as i32, - left: Some(Box::new(iceberg_condition_root)), - right: Some(Box::new(iceberg_predicate)), - }, - ))), - } + iceberg_condition_root = + IcebergPredicate::and(iceberg_condition_root, iceberg_predicate); } None => ignored_conjunctions.push(rw_condition), } } - ( + Some(( + iceberg_condition_root, Condition { conjunctions: ignored_conjunctions, }, - Some(iceberg_condition_root), - ) + )) } let schema = self.schema(); From 08b36761009ffeb8a94a3c2c1787fb662b068070 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 08:45:46 +0800 Subject: [PATCH 15/42] Revert "interim commit: use iceberg proto" This reverts commit 0bef41b8f77e3d94587f7dd95ba8bd281d7e757f. --- .../plan_node/logical_iceberg_scan.rs | 354 ++++++------------ 1 file changed, 110 insertions(+), 244 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 9f2c05f6e7b91..84dee23bb3c0c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -15,20 +15,11 @@ use std::rc::Rc; use chrono::Datelike; +use iceberg::expr::{Predicate as IcebergPredicate, Reference}; +use iceberg::spec::Datum as IcebergDatum; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::types::{DataType, Decimal, ScalarImpl}; -use risingwave_common::util::value_encoding::DatumToProtoExt; -use risingwave_pb::batch_plan::iceberg_binary_predicate::IcebergBinaryExprType; -use risingwave_pb::batch_plan::iceberg_predicate::PredicateExpr; -use risingwave_pb::batch_plan::iceberg_ref_and_value_predicate::IcebergRefAndValueType; -use risingwave_pb::batch_plan::iceberg_ref_predicate::IcebergRefType; -use risingwave_pb::batch_plan::iceberg_unary_predicate::IcebergUnaryExprType; -use risingwave_pb::batch_plan::{ - IcebergBinaryExprType, IcebergBinaryPredicate, IcebergDatum, IcebergPredicate, - IcebergRefAndValuePredicate, IcebergRefAndValueType, IcebergSetPredicate, IcebergUnaryExprType, - IcebergUnaryPredicate, -}; +use risingwave_common::types::{Decimal, ScalarImpl}; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; @@ -38,7 +29,7 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; -use crate::expr::{ExprImpl, ExprType, InputRef, Literal}; +use crate::expr::{ExprImpl, ExprType, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ @@ -119,7 +110,7 @@ impl ExprVisitable for LogicalIcebergScan {} impl PredicatePushdown for LogicalIcebergScan { /// NOTE(kwannoel): /// 1. We expect it to be constant folded - /// 2. We don't convert `inputRefs` of type boolean directly to `IcebergPredicates`. + /// 2. We don't convert `inputRefs` of type boolean directly to IcebergPredicates. /// 3. The leaf nodes are always logical comparison operators: /// `Equal`, `NotEqual`, `GreaterThan`, /// `GreaterThanOrEqual`, `LessThan`, `LessThanOrEqual`. @@ -131,60 +122,39 @@ impl PredicatePushdown for LogicalIcebergScan { _ctx: &mut PredicatePushdownContext, ) -> PlanRef { fn rw_literal_to_iceberg_datum(literal: &Literal) -> Option { - let datum @ Some(scalar) = literal.get_data() else { + let Some(scalar) = literal.get_data() else { return None; }; - match scalar { - ScalarImpl::Bool(b) => Some(IcebergDatum { - datatype: DataType::Boolean.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Int32(i) => Some(IcebergDatum { - datatype: DataType::Int32.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Int64(i) => Some(IcebergDatum { - datatype: DataType::Int64.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Float32(f) => Some(IcebergDatum { - datatype: DataType::Float32.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Float64(f) => Some(IcebergDatum { - datatype: DataType::Float64.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), + ScalarImpl::Bool(b) => Some(IcebergDatum::bool(*b)), + ScalarImpl::Int32(i) => Some(IcebergDatum::int(*i)), + ScalarImpl::Int64(i) => Some(IcebergDatum::long(*i)), + ScalarImpl::Float32(f) => Some(IcebergDatum::float(*f)), + ScalarImpl::Float64(f) => Some(IcebergDatum::double(*f)), ScalarImpl::Decimal(d) => { let Decimal::Normalized(d) = d else { return None; }; - Some(IcebergDatum { - datatype: DataType::Decimal.to_protobuf().into(), - value: datum.to_protobuf().into(), - }) + let Ok(d) = IcebergDatum::decimal(*d) else { + return None; + }; + Some(d) + } + ScalarImpl::Date(d) => { + let Ok(datum) = IcebergDatum::date_from_ymd(d.0.year(), d.0.month(), d.0.day()) + else { + return None; + }; + Some(datum) } - ScalarImpl::Date(d) => Some(IcebergDatum { - datatype: DataType::Date.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Timestamp(t) => Some(IcebergDatum { - datatype: DataType::Timestamp.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Timestamptz(t) => Some(IcebergDatum { - datatype: DataType::Timestamptz.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Utf8(s) => Some(IcebergDatum { - datatype: DataType::Varchar.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), - ScalarImpl::Bytea(b) => Some(IcebergDatum { - datatype: DataType::Bytea.to_protobuf().into(), - value: datum.to_protobuf().into(), - }), + ScalarImpl::Timestamp(t) => Some(IcebergDatum::timestamp_nanos( + t.0.and_utc().timestamp_nanos_opt()?, + )), + ScalarImpl::Timestamptz(t) => { + Some(IcebergDatum::timestamptz_micros(t.timestamp_micros())) + } + ScalarImpl::Utf8(s) => Some(IcebergDatum::string(s)), + ScalarImpl::Bytea(b) => Some(IcebergDatum::binary(b.clone())), _ => None, } } @@ -194,210 +164,133 @@ impl PredicatePushdown for LogicalIcebergScan { fields: &[Field], ) -> Option { match expr { - ExprImpl::Literal(l) => { - if let Some(ScalarImpl::Bool(b)) = l.get_data() { - Some(IcebergPredicate { - predicate_expr: Some(PredicateExpr::Boolean(*b)), - }) - } else { - None + ExprImpl::Literal(l) => match l.get_data() { + Some(ScalarImpl::Bool(b)) => { + if *b { + Some(IcebergPredicate::AlwaysTrue) + } else { + Some(IcebergPredicate::AlwaysFalse) + } } - } + _ => None, + }, ExprImpl::FunctionCall(f) => { - fn create_binary_predicate( - expr_type: IcebergBinaryExprType, - left: &ExprImpl, - right: &ExprImpl, - fields: &[Field], - ) -> Option { - let left = rw_expr_to_iceberg_predicate(left, fields)?; - let right = rw_expr_to_iceberg_predicate(right, fields)?; - Some(IcebergPredicate { - predicate_expr: Some(PredicateExpr::BinaryPredicate(Box::new( - IcebergBinaryPredicate { - expr_type: expr_type as i32, - left: Some(Box::new(left)), - right: Some(Box::new(right)), - }, - ))), - }) - } - - fn create_ref_predicate( - expr_type: IcebergRefType, - reference: &InputRef, - fields: &[Field], - ) -> Option { - let reference = fields[reference.index].name.clone(); - Some(IcebergPredicate { - predicate_expr: Some(PredicateExpr::RefPredicate( - IcebergRefPredicate { - expr_type: expr_type as i32, - reference, - }, - )), - }) - } - - fn create_ref_and_value_predicate( - expr_type: IcebergRefAndValueType, - reference: &InputRef, - arg: &Literal, - fields: &[Field], - ) -> Option { - let reference = fields[reference.index].name.clone(); - let arg = rw_literal_to_iceberg_datum(arg)?; - Some(IcebergPredicate { - predicate_expr: Some(PredicateExpr::RefAndValuePredicate( - IcebergRefAndValuePredicate { - expr_type: expr_type as i32, - reference, - value: Some(arg), - }, - )), - }) - } - let args = f.inputs(); match f.func_type() { ExprType::Not => { let arg = rw_expr_to_iceberg_predicate(&args[0], fields)?; - Some(IcebergPredicate { - predicate_expr: Some(PredicateExpr::UnaryPredicate(Box::new( - IcebergUnaryPredicate { - expr_type: IcebergUnaryExprType::Not as i32, - arg: Some(Box::new(arg)), - }, - ))), - }) + Some(IcebergPredicate::negate(arg)) + } + ExprType::And => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; + Some(IcebergPredicate::and(arg0, arg1)) + } + ExprType::Or => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; + Some(IcebergPredicate::or(arg0, arg1)) } - ExprType::And => create_binary_predicate( - IcebergBinaryExprType::And, - &args[0], - &args[1], - fields, - ), - ExprType::Or => create_binary_predicate( - IcebergBinaryExprType::Or, - &args[0], - &args[1], - fields, - ), ExprType::Equal => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Eq, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.equal_to(datum)) } _ => None, }, ExprType::NotEqual => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Ne, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.not_equal_to(datum)) } _ => None, }, ExprType::GreaterThan => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Gt, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Le, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) } _ => None, }, ExprType::GreaterThanOrEqual => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Ge, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Lt, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) } _ => None, }, ExprType::LessThan => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Lt, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Ge, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) } _ => None, }, ExprType::LessThanOrEqual => match [&args[0], &args[1]] { [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Le, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) } [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - create_ref_and_value_predicate( - IcebergRefAndValueType::Gt, - lhs, - rhs, - fields, - ) + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) } _ => None, }, ExprType::IsNull => match &args[0] { - ExprImpl::InputRef(reference) => { - create_ref_predicate(IcebergRefType::IsNull, reference, fields) + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_null()) } _ => None, }, - ExprType::IsNotNull => match &args[0] { - ExprImpl::InputRef(reference) => { - create_ref_predicate(IcebergRefType::IsNotNull, reference, fields) + ExprType::IsNotNull => match &args[1] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_not_null()) } _ => None, }, - ExprType::In => match &args[0] { + ExprType::In => match &args[1] { ExprImpl::InputRef(lhs) => { - let reference = fields[lhs.index].name.clone(); + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); let mut datums = Vec::with_capacity(args.len() - 1); for arg in &args[1..] { if let ExprImpl::Literal(l) = arg { @@ -410,15 +303,7 @@ impl PredicatePushdown for LogicalIcebergScan { return None; } } - Some(IcebergPredicate { - predicate_expr: Some(PredicateExpr::SetPredicate( - IcebergSetPredicate { - expr_type: IcebergRefAndValueType::In as i32, - reference, - values: datums, - }, - )), - }) + Some(reference.is_in(datums)) } _ => None, }, @@ -431,40 +316,21 @@ impl PredicatePushdown for LogicalIcebergScan { fn rw_predicate_to_iceberg_predicate( predicate: Condition, fields: &[Field], - ) -> (Option<(IcebergPredicate, Condition)>) { + ) -> IcebergPredicate { if predicate.always_true() { - return None; + return IcebergPredicate::AlwaysTrue; } let mut conjunctions = predicate.conjunctions; let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); - let mut iceberg_condition_root; - loop { - let Some(rw_condition_root) = conjunctions.pop() else { - return None; - }; - match rw_expr_to_iceberg_predicate(&rw_condition_root, fields) { - Some(iceberg_predicate) => { - iceberg_condition_root = iceberg_predicate; - break; - } - None => ignored_conjunctions.push(rw_condition_root), - } - } + let rw_condition_root = conjunctions.pop().unwrap(); + let iceberg_condition_root = rw_expr_to_iceberg_predicate(&rw_condition_root, fields); for rw_condition in conjunctions { match rw_expr_to_iceberg_predicate(&rw_condition, fields) { - Some(iceberg_predicate) => { - iceberg_condition_root = - IcebergPredicate::and(iceberg_condition_root, iceberg_predicate); - } - None => ignored_conjunctions.push(rw_condition), + Some(iceberg_predicate) => ignored_conjunctions.push(rw_condition), + None => {} } } - Some(( - iceberg_condition_root, - Condition { - conjunctions: ignored_conjunctions, - }, - )) + todo!() } let schema = self.schema(); From 31fa17f5397ca4d3a4795414a6c5ee8e74b73102 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 08:55:45 +0800 Subject: [PATCH 16/42] Revert "add predicate proto" This reverts commit eb4971778647fe5ca2152d511005b8b8ad0b3873. --- proto/batch_plan.proto | 100 ----------------------------------------- 1 file changed, 100 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index f3980f5e5a765..ac88a913130c7 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -66,106 +66,6 @@ message SourceNode { map secret_refs = 6; } -//enum IcebergPredicateType { -// ICEBERG_PREDICATE_UNSPECIFIED = 0; -// ICEBERG_PREDICATE_TRUE = 1; -// ICEBERG_PREDICATE_FALSE = 2; -// ICEBERG_PREDICATE_EQ = 3; -// ICEBERG_PREDICATE_NE = 4; -// ICEBERG_PREDICATE_LT = 5; -// ICEBERG_PREDICATE_LE = 6; -// ICEBERG_PREDICATE_GT = 7; -// ICEBERG_PREDICATE_GE = 8; -// ICEBERG_PREDICATE_IS_NULL = 9; -// ICEBERG_PREDICATE_IS_NOT_NULL = 10; -// ICEBERG_PREDICATE_IN = 11; -// ICEBERG_PREDICATE_NOT_IN = 12; -// ICEBERG_PREDICATE_AND = 13; -// ICEBERG_PREDICATE_OR = 14; -// ICEBERG_PREDICATE_NOT = 15; -//} - -message IcebergBinaryPredicate { - enum IcebergBinaryExprType { - ICEBERG_BINARY_EXPR_TYPE_UNSPECIFIED = 0; - ICEBERG_BINARY_EXPR_TYPE_AND = 1; - ICEBERG_BINARY_EXPR_TYPE_OR = 2; - } - IcebergBinaryExprType expr_type = 1; - IcebergPredicate left = 2; - IcebergPredicate right = 3; -} - -message IcebergUnaryPredicate { - enum IcebergUnaryExprType { - ICEBERG_UNARY_EXPR_TYPE_UNSPECIFIED = 0; - ICEBERG_UNARY_EXPR_TYPE_NOT = 1; - } - IcebergUnaryExprType expr_type = 1; - IcebergPredicate arg = 2; -} - -message IcebergDatum { - data.DataType datatype = 1; - data.Datum value = 2; -} - -message IcebergRefAndValuePredicate { - enum IcebergRefAndValueType { - ICEBERG_REF_AND_VALUE_TYPE_UNSPECIFIED = 0; - ICEBERG_REF_AND_VALUE_TYPE_EQ = 1; - ICEBERG_REF_AND_VALUE_TYPE_NE = 2; - ICEBERG_REF_AND_VALUE_TYPE_LT = 3; - ICEBERG_REF_AND_VALUE_TYPE_LE = 4; - ICEBERG_REF_AND_VALUE_TYPE_GT = 5; - ICEBERG_REF_AND_VALUE_TYPE_GE = 6; - ICEBERG_REF_AND_VALUE_TYPE_IS_NULL = 7; - ICEBERG_REF_AND_VALUE_TYPE_IS_NOT_NULL = 8; - ICEBERG_REF_AND_VALUE_TYPE_IN = 9; - } - IcebergRefAndValueType expr_type = 1; - string reference = 2; - IcebergDatum value = 3; -} - -message IcebergRefPredicate { - enum IcebergRefType { - ICEBERG_REF_TYPE_UNSPECIFIED = 0; - ICEBERG_REF_TYPE_EQ = 1; - ICEBERG_REF_TYPE_NE = 2; - ICEBERG_REF_TYPE_LT = 3; - ICEBERG_REF_TYPE_LE = 4; - ICEBERG_REF_TYPE_GT = 5; - ICEBERG_REF_TYPE_GE = 6; - ICEBERG_REF_TYPE_IS_NULL = 7; - ICEBERG_REF_TYPE_IS_NOT_NULL = 8; - ICEBERG_REF_TYPE_IN = 9; - } - IcebergRefType expr_type = 1; - string reference = 2; -} - -message IcebergSetPredicate { - enum IcebergSetType { - ICEBERG_SET_TYPE_UNSPECIFIED = 0; - ICEBERG_SET_TYPE_IN = 1; - } - IcebergSetType expr_type = 1; - string reference = 2; - repeated IcebergDatum values = 3; -} - -message IcebergPredicate { - oneof predicate_expr { - bool boolean = 1; - IcebergBinaryPredicate binary_predicate = 3; - IcebergUnaryPredicate unary_predicate = 4; - IcebergRefPredicate ref_predicate = 5; - IcebergRefAndValuePredicate ref_and_value_predicate = 6; - IcebergSetPredicate set_predicate = 7; - } -} - message IcebergScanNode { repeated plan_common.ColumnCatalog columns = 1; map with_properties = 2; From d9e4a9eec8a1737700aab8ba4feebb86bead1eba Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 08:58:01 +0800 Subject: [PATCH 17/42] Revert "interim commit: add iceberg_predicate to batch" This reverts commit 4859705e496e77eb9005316e87a6fec1844caf76. --- proto/batch_plan.proto | 1 - .../src/optimizer/plan_node/batch_iceberg_scan.rs | 13 +++---------- .../src/optimizer/plan_node/logical_iceberg_scan.rs | 3 +-- 3 files changed, 4 insertions(+), 13 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index ac88a913130c7..b46230b2438d6 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -71,7 +71,6 @@ message IcebergScanNode { map with_properties = 2; repeated bytes split = 3; map secret_refs = 4; - IcebergPredicate predicate = 5; } message FileScanNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs index 3a1f753e78c18..4333fcaa3e90a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs @@ -16,7 +16,7 @@ use std::rc::Rc; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; -use risingwave_pb::batch_plan::{IcebergPredicate, IcebergScanNode}; +use risingwave_pb::batch_plan::IcebergScanNode; use risingwave_sqlparser::ast::AsOf; use super::batch::prelude::*; @@ -33,11 +33,10 @@ use crate::optimizer::property::{Distribution, Order}; pub struct BatchIcebergScan { pub base: PlanBase, pub core: generic::Source, - pub iceberg_predicate: IcebergPredicate, } impl BatchIcebergScan { - pub fn new(core: generic::Source, iceberg_predicate: IcebergPredicate) -> Self { + pub fn new(core: generic::Source) -> Self { let base = PlanBase::new_batch_with_core( &core, // Use `Single` by default, will be updated later with `clone_with_dist`. @@ -45,11 +44,7 @@ impl BatchIcebergScan { Order::any(), ); - Self { - base, - core, - iceberg_predicate, - } + Self { base, core } } pub fn column_names(&self) -> Vec<&str> { @@ -67,7 +62,6 @@ impl BatchIcebergScan { Self { base, core: self.core.clone(), - iceberg_predicate: self.iceberg_predicate.clone(), } } @@ -115,7 +109,6 @@ impl ToBatchPb for BatchIcebergScan { with_properties, split: vec![], secret_refs, - predicate: Some(self.iceberg_predicate.clone()), }) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 84dee23bb3c0c..091f46789217e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -342,8 +342,7 @@ impl PredicatePushdown for LogicalIcebergScan { impl ToBatch for LogicalIcebergScan { fn to_batch(&self) -> Result { - let plan: PlanRef = - BatchIcebergScan::new(self.core.clone(), self.iceberg_predicate.clone()).into(); + let plan: PlanRef = BatchIcebergScan::new(self.core.clone()).into(); Ok(plan) } } From 9580498422c8b0c3d4ebeac5f176019c652874f4 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 15:18:22 +0800 Subject: [PATCH 18/42] use iceberg predicate in logical_iceberg_scan fields --- .../plan_node/logical_iceberg_scan.rs | 81 ++++++++++++++++--- 1 file changed, 70 insertions(+), 11 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 091f46789217e..588af148c18d1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -15,6 +15,7 @@ use std::rc::Rc; use chrono::Datelike; +use educe::Educe; use iceberg::expr::{Predicate as IcebergPredicate, Reference}; use iceberg::spec::Datum as IcebergDatum; use pretty_xmlish::{Pretty, XmlNode}; @@ -39,10 +40,14 @@ use crate::optimizer::plan_node::{ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalIcebergScan` is only used by batch queries. At the beginning of the batch query optimization, `LogicalSource` with a iceberg property would be converted into a `LogicalIcebergScan`. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Educe, Debug, Clone, PartialEq)] +#[educe(Eq, Hash)] pub struct LogicalIcebergScan { pub base: PlanBase, pub core: generic::Source, + /// TODO(kwannoel): If we support plan sharing, we can't just ignore the predicate. + #[educe(Hash(ignore))] + pub predicate: IcebergPredicate, } impl LogicalIcebergScan { @@ -54,7 +59,11 @@ impl LogicalIcebergScan { assert!(logical_source.output_exprs.is_none()); - LogicalIcebergScan { base, core } + LogicalIcebergScan { + base, + core, + predicate: IcebergPredicate::AlwaysTrue, + } } pub fn source_catalog(&self) -> Option> { @@ -70,7 +79,21 @@ impl LogicalIcebergScan { .collect(); let base = PlanBase::new_logical_with_core(&core); - LogicalIcebergScan { base, core } + LogicalIcebergScan { + base, + core, + predicate: self.predicate.clone(), + } + } + + pub fn clone_with_predicate(&self, predicate: IcebergPredicate) -> Self { + let base = PlanBase::new_logical_with_core(&self.core); + let predicate = predicate.and(self.predicate.clone()); + LogicalIcebergScan { + base, + core: self.core.clone(), + predicate, + } } } @@ -316,27 +339,63 @@ impl PredicatePushdown for LogicalIcebergScan { fn rw_predicate_to_iceberg_predicate( predicate: Condition, fields: &[Field], - ) -> IcebergPredicate { + ) -> (IcebergPredicate, Condition) { if predicate.always_true() { - return IcebergPredicate::AlwaysTrue; + return (IcebergPredicate::AlwaysTrue, predicate); } + let mut conjunctions = predicate.conjunctions; let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); - let rw_condition_root = conjunctions.pop().unwrap(); - let iceberg_condition_root = rw_expr_to_iceberg_predicate(&rw_condition_root, fields); + + let mut iceberg_condition_root = None; + while let Some(conjunction) = conjunctions.pop() { + match rw_expr_to_iceberg_predicate(&conjunction, fields) { + iceberg_predicate @ Some(_) => { + iceberg_condition_root = iceberg_predicate; + break; + } + None => { + ignored_conjunctions.push(conjunction); + continue; + } + } + } + + let mut iceberg_condition_root = match iceberg_condition_root { + Some(p) => p, + None => { + return ( + IcebergPredicate::AlwaysTrue, + Condition { + conjunctions: ignored_conjunctions, + }, + ) + } + }; + for rw_condition in conjunctions { match rw_expr_to_iceberg_predicate(&rw_condition, fields) { - Some(iceberg_predicate) => ignored_conjunctions.push(rw_condition), - None => {} + Some(iceberg_predicate) => { + iceberg_condition_root = iceberg_condition_root.and(iceberg_predicate) + } + None => ignored_conjunctions.push(rw_condition), } } - todo!() + ( + iceberg_condition_root, + Condition { + conjunctions: ignored_conjunctions, + }, + ) } let schema = self.schema(); let fields = &schema.fields; + let (iceberg_predicate, rw_predicate) = + rw_predicate_to_iceberg_predicate(predicate, fields); // No pushdown. - LogicalFilter::create(self.clone().into(), predicate) + let this = self.clone_with_predicate(iceberg_predicate); + LogicalFilter::create(this.into(), rw_predicate) } } From 631c28feb366d410f6af85cc28ff485da39d94dc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 15:22:47 +0800 Subject: [PATCH 19/42] add to batch --- .../optimizer/plan_node/batch_iceberg_scan.rs | 16 +++++++++++++--- .../optimizer/plan_node/logical_iceberg_scan.rs | 2 +- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs index 4333fcaa3e90a..b491e4815293c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs @@ -14,6 +14,8 @@ use std::rc::Rc; +use educe::Educe; +use iceberg::expr::Predicate as IcebergPredicate; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::IcebergScanNode; @@ -29,14 +31,17 @@ use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::property::{Distribution, Order}; -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Educe, Debug, Clone, PartialEq)] +#[educe(Eq, Hash)] pub struct BatchIcebergScan { pub base: PlanBase, pub core: generic::Source, + #[educe(Hash(ignore))] + pub predicate: IcebergPredicate, } impl BatchIcebergScan { - pub fn new(core: generic::Source) -> Self { + pub fn new(core: generic::Source, predicate: IcebergPredicate) -> Self { let base = PlanBase::new_batch_with_core( &core, // Use `Single` by default, will be updated later with `clone_with_dist`. @@ -44,7 +49,11 @@ impl BatchIcebergScan { Order::any(), ); - Self { base, core } + Self { + base, + core, + predicate, + } } pub fn column_names(&self) -> Vec<&str> { @@ -62,6 +71,7 @@ impl BatchIcebergScan { Self { base, core: self.core.clone(), + predicate: self.predicate.clone(), } } diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 588af148c18d1..d2c1a8d5aba02 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -401,7 +401,7 @@ impl PredicatePushdown for LogicalIcebergScan { impl ToBatch for LogicalIcebergScan { fn to_batch(&self) -> Result { - let plan: PlanRef = BatchIcebergScan::new(self.core.clone()).into(); + let plan: PlanRef = BatchIcebergScan::new(self.core.clone(), self.predicate.clone()).into(); Ok(plan) } } From 77342c01303f3d2cab69636816e5f345de616055 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 15:28:29 +0800 Subject: [PATCH 20/42] build with predicate --- src/connector/src/source/iceberg/mod.rs | 3 +++ src/frontend/src/scheduler/plan_fragmenter.rs | 16 +++++++++++++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index c6f5f08d1b305..e717d604a31a3 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; use anyhow::anyhow; use async_trait::async_trait; use futures_async_stream::for_await; +use iceberg::expr::Predicate as IcebergPredicate; use iceberg::scan::FileScanTask; use iceberg::spec::TableMetadata; use iceberg::table::Table; @@ -189,6 +190,7 @@ impl IcebergSplitEnumerator { schema: Schema, time_traval_info: Option, batch_parallelism: usize, + predicate: IcebergPredicate, ) -> ConnectorResult> { if batch_parallelism == 0 { bail!("Batch parallelism is 0. Cannot split the iceberg files."); @@ -252,6 +254,7 @@ impl IcebergSplitEnumerator { // TODO(kwannoel): add filter here. let scan = table .scan() + .with_filter(predicate) .snapshot_id(snapshot_id) .select(require_names) .build() diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 4db8302bb407c..35eafa8a8be86 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -358,7 +358,10 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } - (ConnectorProperties::Iceberg(prop), SourceFetchParameters::IcebergPredicate(_)) => { + ( + ConnectorProperties::Iceberg(prop), + SourceFetchParameters::IcebergPredicate(predicate), + ) => { let iceberg_enumerator = IcebergSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) .await?; @@ -387,7 +390,12 @@ impl SourceScanInfo { }; let split_info = iceberg_enumerator - .list_splits_batch(fetch_info.schema, time_travel_info, batch_parallelism) + .list_splits_batch( + fetch_info.schema, + time_travel_info, + batch_parallelism, + predicate, + ) .await? .into_iter() .map(SplitImpl::Iceberg) @@ -1103,7 +1111,9 @@ impl BatchPlanFragmenter { return Ok(Some(SourceScanInfo::new(SourceFetchInfo { schema: batch_iceberg_scan.base.schema().clone(), connector: property, - fetch_parameters: SourceFetchParameters::Empty, + fetch_parameters: SourceFetchParameters::IcebergPredicate( + batch_iceberg_scan.predicate.clone(), + ), as_of, }))); } From 06d8f8aa8d37c97c43a0b6e5c4d4dcfa96d36ebb Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 6 Nov 2024 15:31:14 +0800 Subject: [PATCH 21/42] clean --- src/connector/src/source/iceberg/mod.rs | 1 - src/frontend/src/scheduler/plan_fragmenter.rs | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index e717d604a31a3..58faf022322a0 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -251,7 +251,6 @@ impl IcebergSplitEnumerator { let mut position_delete_files = vec![]; let mut data_files = vec![]; let mut equality_delete_files = vec![]; - // TODO(kwannoel): add filter here. let scan = table .scan() .with_filter(predicate) diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 35eafa8a8be86..9cec27601a246 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -270,7 +270,7 @@ impl Query { } #[derive(Debug, Clone)] -enum SourceFetchParameters { +pub enum SourceFetchParameters { IcebergPredicate(IcebergPredicate), KafkaTimebound { lower: Option, From 081dfa6d4dc0ea350938604494b81f15ec96cd5e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 10:28:46 +0800 Subject: [PATCH 22/42] implement distill --- src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs | 1 + src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs index b491e4815293c..1c91d19a9fcc9 100644 --- a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs @@ -88,6 +88,7 @@ impl Distill for BatchIcebergScan { let fields = vec![ ("source", src), ("columns", column_names_pretty(self.schema())), + ("predicate", Pretty::from(self.predicate.to_string())), ]; childless_record("BatchIcebergScan", fields) } diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index d2c1a8d5aba02..609925cbd3d40 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -105,6 +105,7 @@ impl Distill for LogicalIcebergScan { vec![ ("source", src), ("columns", column_names_pretty(self.schema())), + ("predicate", Pretty::from(self.predicate.to_string())), ] } else { vec![] From e7a3609af63da60e1b76b866701730b1f729d335 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 10:30:00 +0800 Subject: [PATCH 23/42] fix warn --- src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 609925cbd3d40..6135d27a5b00f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -19,7 +19,7 @@ use educe::Educe; use iceberg::expr::{Predicate as IcebergPredicate, Reference}; use iceberg::spec::Datum as IcebergDatum; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::Field; use risingwave_common::types::{Decimal, ScalarImpl}; use super::generic::GenericPlanRef; @@ -134,7 +134,7 @@ impl ExprVisitable for LogicalIcebergScan {} impl PredicatePushdown for LogicalIcebergScan { /// NOTE(kwannoel): /// 1. We expect it to be constant folded - /// 2. We don't convert `inputRefs` of type boolean directly to IcebergPredicates. + /// 2. We don't convert `inputRefs` of type boolean directly to `IcebergPredicates`. /// 3. The leaf nodes are always logical comparison operators: /// `Equal`, `NotEqual`, `GreaterThan`, /// `GreaterThanOrEqual`, `LessThan`, `LessThanOrEqual`. From 909045658d0523a2560e327fa27ff0a1c2fc3540 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 11:07:15 +0800 Subject: [PATCH 24/42] add tests --- ci/scripts/e2e-iceberg-sink-v2-test.sh | 1 + .../test_case/iceberg_predicate_pushdown.slt | 75 +++++++++++++++++++ .../test_case/iceberg_predicate_pushdown.toml | 11 +++ 3 files changed, 87 insertions(+) create mode 100644 e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt create mode 100644 e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml diff --git a/ci/scripts/e2e-iceberg-sink-v2-test.sh b/ci/scripts/e2e-iceberg-sink-v2-test.sh index bcb530ae9fdd9..27fc92a789d1b 100755 --- a/ci/scripts/e2e-iceberg-sink-v2-test.sh +++ b/ci/scripts/e2e-iceberg-sink-v2-test.sh @@ -49,6 +49,7 @@ poetry run python main.py -t ./test_case/iceberg_select_empty_table.toml poetry run python main.py -t ./test_case/iceberg_source_equality_delete.toml poetry run python main.py -t ./test_case/iceberg_source_position_delete.toml poetry run python main.py -t ./test_case/iceberg_source_all_delete.toml +poetry run python main.py -t ./test_case/iceberg_predicate_pushdown.toml echo "--- Kill cluster" diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt new file mode 100644 index 0000000000000..ea35a7d21598c --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -0,0 +1,75 @@ +statement ok +set sink_decouple = false; + +statement ok +set streaming_parallelism=4; + +statement ok +CREATE TABLE s1 (i1 int, i2 varchar, i3 varchar); + +statement ok +insert into s1 select * from generate_series(1, 1000); + +statement ok +flush; + +statement ok +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM s1; + +statement ok +CREATE SINK sink1 AS select * from mv1 WITH ( + connector = 'iceberg', + type = 'append-only', + force_append_only = 'true', + database.name = 'demo_db', + table.name = 't1', + catalog.name = 'demo', + catalog.type = 'storage', + warehouse.path = 's3a://icebergdata/demo', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + commit_checkpoint_interval = 1, + create_table_if_not_exists = 'true' +); + +statement ok +CREATE SOURCE iceberg_t1_source +WITH ( + connector = 'iceberg', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + s3.path.style.access = 'true', + catalog.type = 'storage', + warehouse.path = 's3a://icebergdata/demo', + database.name = 'demo_db', + table.name = 't1', +); + +statement ok +flush; + +query I +select count(*) from iceberg_t1_source; +---- +1000 + +query I +explain select count(*) from iceberg_t1_source where i1 > 500; +---- + +query I +explain select count(*) from iceberg_t1_source where i1 > 500 and i1 < 600; +---- + +statement ok +DROP SINK sink1; + +statement ok +DROP SOURCE iceberg_t1_source; + +statement ok +DROP TABLE s1 cascade; diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml new file mode 100644 index 0000000000000..c08dcbb827db1 --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml @@ -0,0 +1,11 @@ +init_sqls = [ + 'CREATE SCHEMA IF NOT EXISTS demo_db', + 'DROP TABLE IF EXISTS demo_db.t1', +] + +slt = 'test_case/iceberg_predicate_pushdown.slt' + +drop_sqls = [ + 'DROP TABLE IF EXISTS demo_db.t1', + 'DROP SCHEMA IF EXISTS demo_db', +] From 9c1c87697e05d40e7a9a2635b965f060079ab97b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 12:31:35 +0800 Subject: [PATCH 25/42] no verbose for wget --- e2e_test/iceberg/start_spark_connect_server.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/iceberg/start_spark_connect_server.sh b/e2e_test/iceberg/start_spark_connect_server.sh index 8f0c2640a1b59..7996899f7a4d1 100755 --- a/e2e_test/iceberg/start_spark_connect_server.sh +++ b/e2e_test/iceberg/start_spark_connect_server.sh @@ -11,7 +11,7 @@ PACKAGES="$PACKAGES,org.apache.spark:spark-connect_2.12:$SPARK_VERSION" SPARK_FILE="spark-${SPARK_VERSION}-bin-hadoop3.tgz" if [ ! -d "spark-${SPARK_VERSION}-bin-hadoop3" ];then - wget https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/$SPARK_FILE + wget --no-verbose https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/$SPARK_FILE tar -xzf $SPARK_FILE --no-same-owner fi From 99f8874641e3e66300a749b96a991f4081bae1f5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 12:35:19 +0800 Subject: [PATCH 26/42] more tests --- .../test_case/iceberg_predicate_pushdown.slt | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt index ea35a7d21598c..8112597311e02 100644 --- a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -60,10 +60,26 @@ select count(*) from iceberg_t1_source; query I explain select count(*) from iceberg_t1_source where i1 > 500; ---- + BatchSimpleAgg { aggs: [sum0(count)] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [count] } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1], predicate: i1 > 500 } query I explain select count(*) from iceberg_t1_source where i1 > 500 and i1 < 600; ---- + BatchSimpleAgg { aggs: [sum0(count)] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [count] } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1], predicate: i1 > 500 and i1 < 600 } + +query I +explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5); +---- + BatchSimpleAgg { aggs: [sum0(count)] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [count] } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1], predicate: i1 > 500 and i1 < 600 } statement ok DROP SINK sink1; From a4e03c19ee417967a9e7a7955d1856233ea3a926 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 15:35:36 +0800 Subject: [PATCH 27/42] check results --- .../test_case/iceberg_predicate_pushdown.slt | 40 ++++++++++++------- 1 file changed, 26 insertions(+), 14 deletions(-) diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt index 8112597311e02..c1a95573f3982 100644 --- a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -4,6 +4,9 @@ set sink_decouple = false; statement ok set streaming_parallelism=4; +statement ok +drop table if exists s1 cascade; + statement ok CREATE TABLE s1 (i1 int, i2 varchar, i3 varchar); @@ -34,6 +37,9 @@ CREATE SINK sink1 AS select * from mv1 WITH ( create_table_if_not_exists = 'true' ); +statement ok +drop source if exists iceberg_t1_source; + statement ok CREATE SOURCE iceberg_t1_source WITH ( @@ -58,28 +64,34 @@ select count(*) from iceberg_t1_source; 1000 query I -explain select count(*) from iceberg_t1_source where i1 > 500; +select * from iceberg_t1_source where i1 > 990; +---- + +query I +explain select * from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580; ---- - BatchSimpleAgg { aggs: [sum0(count)] } - └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [count] } - └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1], predicate: i1 > 500 } + BatchExchange { order: [], dist: Single } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: (((((i1 = 580) AND (i1 > 500)) AND (i1 < 600)) AND (i1 >= 550)) AND (i1 <= 590)) AND (i1 != 570) } query I -explain select count(*) from iceberg_t1_source where i1 > 500 and i1 < 600; +select * from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580; ---- - BatchSimpleAgg { aggs: [sum0(count)] } - └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [count] } - └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1], predicate: i1 > 500 and i1 < 600 } +580 query I explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5); ---- - BatchSimpleAgg { aggs: [sum0(count)] } - └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [count] } - └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1], predicate: i1 > 500 and i1 < 600 } + BatchExchange { order: [], dist: Single } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: i1 IN (5, 4, 1, 3, 2) } + +query I +explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5) order by i1; +---- +1 +2 +3 +4 +5 statement ok DROP SINK sink1; From aede6a9e3f406071ff60eb6362c3636403f1fbda Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 15:35:47 +0800 Subject: [PATCH 28/42] fix bugs --- src/batch/src/executor/iceberg_scan.rs | 1 + src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/batch/src/executor/iceberg_scan.rs b/src/batch/src/executor/iceberg_scan.rs index 229cff77e9b9c..ec6b4c4ceabcb 100644 --- a/src/batch/src/executor/iceberg_scan.rs +++ b/src/batch/src/executor/iceberg_scan.rs @@ -194,6 +194,7 @@ impl BoxedExecutorBuilder for IcebergScanExecutorBuilder { ); let config = ConnectorProperties::extract(options_with_secret.clone(), false)?; + assert_eq!(source_node.split.len(), 1, "Iceberg source should have one split!"); let split_list = source_node .split .iter() diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 6135d27a5b00f..f2698448d3315 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -303,7 +303,7 @@ impl PredicatePushdown for LogicalIcebergScan { } _ => None, }, - ExprType::IsNotNull => match &args[1] { + ExprType::IsNotNull => match &args[0] { ExprImpl::InputRef(lhs) => { let column_name = &fields[lhs.index].name; let reference = Reference::new(column_name); @@ -311,7 +311,7 @@ impl PredicatePushdown for LogicalIcebergScan { } _ => None, }, - ExprType::In => match &args[1] { + ExprType::In => match &args[0] { ExprImpl::InputRef(lhs) => { let column_name = &fields[lhs.index].name; let reference = Reference::new(column_name); From 1a8b370cc9f9aef56965f8563f718162612ba9d5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 7 Nov 2024 17:01:27 +0800 Subject: [PATCH 29/42] explain source plan, maybe schema malformed --- .../test_case/iceberg_predicate_pushdown.slt | 17 +++++++++++++++++ src/connector/src/source/iceberg/mod.rs | 11 +++++++---- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt index c1a95573f3982..2d0d9ccaeb715 100644 --- a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -40,6 +40,23 @@ CREATE SINK sink1 AS select * from mv1 WITH ( statement ok drop source if exists iceberg_t1_source; +query I +explain CREATE SOURCE iceberg_t1_source +WITH ( + connector = 'iceberg', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + s3.path.style.access = 'true', + catalog.type = 'storage', + warehouse.path = 's3a://icebergdata/demo', + database.name = 'demo_db', + table.name = 't1', +); +---- + + statement ok CREATE SOURCE iceberg_t1_source WITH ( diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 58faf022322a0..3513ac7206dd3 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -305,10 +305,13 @@ impl IcebergSplitEnumerator { .files .push(data_files[split_num * split_size + i].clone()); } - Ok(splits - .into_iter() - .filter(|split| !split.files.is_empty()) - .collect_vec()) + let splits = splits.into_iter().map(|split| split).collect_vec(); + + if splits.is_empty() { + bail!("No splits found for the iceberg table"); + } + + Ok(splits) } /// The required field names are the intersection of the output shema and the equality delete columns. From 0f561ac65432d816aa3d34e8329bc8b4de54417e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 12:21:29 +0800 Subject: [PATCH 30/42] fix tests --- .../test_case/iceberg_predicate_pushdown.slt | 48 +++++++++++-------- src/connector/src/source/iceberg/mod.rs | 3 ++ 2 files changed, 30 insertions(+), 21 deletions(-) diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt index 2d0d9ccaeb715..4cee28f4e0ea6 100644 --- a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -11,7 +11,10 @@ statement ok CREATE TABLE s1 (i1 int, i2 varchar, i3 varchar); statement ok -insert into s1 select * from generate_series(1, 1000); +insert into s1 select x, 'some str', 'another str' from generate_series(1, 500) t(x); + +statement ok +insert into s1 select x, null as y, null as z from generate_series(501, 1000) t(x); statement ok flush; @@ -40,23 +43,6 @@ CREATE SINK sink1 AS select * from mv1 WITH ( statement ok drop source if exists iceberg_t1_source; -query I -explain CREATE SOURCE iceberg_t1_source -WITH ( - connector = 'iceberg', - s3.endpoint = 'http://127.0.0.1:9301', - s3.region = 'us-east-1', - s3.access.key = 'hummockadmin', - s3.secret.key = 'hummockadmin', - s3.path.style.access = 'true', - catalog.type = 'storage', - warehouse.path = 's3a://icebergdata/demo', - database.name = 'demo_db', - table.name = 't1', -); ----- - - statement ok CREATE SOURCE iceberg_t1_source WITH ( @@ -75,14 +61,29 @@ WITH ( statement ok flush; +query I +select * from iceberg_t1_source order by i1 limit 1; +---- +1 some str another str + query I select count(*) from iceberg_t1_source; ---- 1000 query I -select * from iceberg_t1_source where i1 > 990; +select * from iceberg_t1_source where i1 > 990 order by i1; ---- +991 NULL NULL +992 NULL NULL +993 NULL NULL +994 NULL NULL +995 NULL NULL +996 NULL NULL +997 NULL NULL +998 NULL NULL +999 NULL NULL +1000 NULL NULL query I explain select * from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580; @@ -91,7 +92,7 @@ explain select * from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 55 └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: (((((i1 = 580) AND (i1 > 500)) AND (i1 < 600)) AND (i1 >= 550)) AND (i1 <= 590)) AND (i1 != 570) } query I -select * from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580; +select i1 from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580; ---- 580 @@ -102,7 +103,7 @@ explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5); └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: i1 IN (5, 4, 1, 3, 2) } query I -explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5) order by i1; +select i1 from iceberg_t1_source where i1 in (1, 2, 3, 4, 5) order by i1; ---- 1 2 @@ -110,6 +111,11 @@ explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5) order by i1; 4 5 +query I +select count(*), i2, i3 from iceberg_t1_source where i2 = 'some str' and i3 = 'another str' group by i2, i3; +---- +500 some str another str + statement ok DROP SINK sink1; diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 3513ac7206dd3..73dff4b7ac270 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -248,6 +248,9 @@ impl IcebergSplitEnumerator { let require_names = Self::get_require_field_names(&table, snapshot_id, &schema).await?; + let table_schema = table.metadata().current_schema(); + tracing::debug!("iceberg_table_schema: {:?}", table_schema); + let mut position_delete_files = vec![]; let mut data_files = vec![]; let mut equality_delete_files = vec![]; From 7685a97142cb04c664572a0567120d4dd9eb152f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 12:52:21 +0800 Subject: [PATCH 31/42] fmt --- src/batch/src/executor/iceberg_scan.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/batch/src/executor/iceberg_scan.rs b/src/batch/src/executor/iceberg_scan.rs index ec6b4c4ceabcb..64ea33b91817e 100644 --- a/src/batch/src/executor/iceberg_scan.rs +++ b/src/batch/src/executor/iceberg_scan.rs @@ -194,7 +194,11 @@ impl BoxedExecutorBuilder for IcebergScanExecutorBuilder { ); let config = ConnectorProperties::extract(options_with_secret.clone(), false)?; - assert_eq!(source_node.split.len(), 1, "Iceberg source should have one split!"); + assert_eq!( + source_node.split.len(), + 1, + "Iceberg source should have one split!" + ); let split_list = source_node .split .iter() From ddb7d7b5b655139e930c6ee7c6d5b3a2b71a03a8 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 13:22:40 +0800 Subject: [PATCH 32/42] increase timeout --- ci/workflows/main-cron.yml | 3 +-- ci/workflows/pull-request.yml | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 3f3cd705f09f9..e8a0fa32f1010 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -414,14 +414,13 @@ steps: depends_on: - "build" - "build-other" - plugins: - docker-compose#v5.1.0: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 7 + timeout_in_minutes: 9 retry: *auto-retry - label: "end-to-end iceberg sink v2 test (release)" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 2991397dfb1c5..3b315a7e5c8ff 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -255,7 +255,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 15 + timeout_in_minutes: 17 retry: *auto-retry - label: "end-to-end iceberg sink v2 test" From 4007509ff095f0213ac74733d104220307632a91 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 13:34:56 +0800 Subject: [PATCH 33/42] no need double assert --- src/batch/src/executor/iceberg_scan.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/batch/src/executor/iceberg_scan.rs b/src/batch/src/executor/iceberg_scan.rs index 64ea33b91817e..229cff77e9b9c 100644 --- a/src/batch/src/executor/iceberg_scan.rs +++ b/src/batch/src/executor/iceberg_scan.rs @@ -194,11 +194,6 @@ impl BoxedExecutorBuilder for IcebergScanExecutorBuilder { ); let config = ConnectorProperties::extract(options_with_secret.clone(), false)?; - assert_eq!( - source_node.split.len(), - 1, - "Iceberg source should have one split!" - ); let split_list = source_node .split .iter() From 1051e091a0e40c704bc98836a323a7f249984604 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 13:46:34 +0800 Subject: [PATCH 34/42] docs --- .../src/optimizer/plan_node/logical_iceberg_scan.rs | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index f2698448d3315..a57d536f4195a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -132,14 +132,11 @@ impl ExprRewritable for LogicalIcebergScan {} impl ExprVisitable for LogicalIcebergScan {} impl PredicatePushdown for LogicalIcebergScan { - /// NOTE(kwannoel): - /// 1. We expect it to be constant folded - /// 2. We don't convert `inputRefs` of type boolean directly to `IcebergPredicates`. - /// 3. The leaf nodes are always logical comparison operators: - /// `Equal`, `NotEqual`, `GreaterThan`, - /// `GreaterThanOrEqual`, `LessThan`, `LessThanOrEqual`. - /// 4. For leaf nodes, their LHS is always an `inputRef` - /// and their RHS is always a `Literal` to be compatible with Iceberg. + /// NOTE(kwannoel): We do predicate pushdown to the iceberg-sdk here. + /// Currently we depend on third-party tools to compact iceberg tables and provide zone-map. + /// zone-map is used to evaluate predicates on iceberg tables. + /// Without zone-map, iceberg-sdk will still apply the predicate on its own. + /// See: https://github.com/apache/iceberg-rust/blob/5c1a9e68da346819072a15327080a498ad91c488/crates/iceberg/src/arrow/reader.rs#L229-L235. fn predicate_pushdown( &self, predicate: Condition, From e9642ec453d0c19276d21fa2d3ee33e92f8acbc5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 15:42:42 +0800 Subject: [PATCH 35/42] fmt --- src/connector/src/source/iceberg/mod.rs | 5 ++++- src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 73dff4b7ac270..5c879dea63f12 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -308,7 +308,10 @@ impl IcebergSplitEnumerator { .files .push(data_files[split_num * split_size + i].clone()); } - let splits = splits.into_iter().map(|split| split).collect_vec(); + let splits = splits + .into_iter() + .filter(|split| !split.files.is_empty()) + .collect_vec(); if splits.is_empty() { bail!("No splits found for the iceberg table"); diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index a57d536f4195a..84a3217245266 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -136,7 +136,7 @@ impl PredicatePushdown for LogicalIcebergScan { /// Currently we depend on third-party tools to compact iceberg tables and provide zone-map. /// zone-map is used to evaluate predicates on iceberg tables. /// Without zone-map, iceberg-sdk will still apply the predicate on its own. - /// See: https://github.com/apache/iceberg-rust/blob/5c1a9e68da346819072a15327080a498ad91c488/crates/iceberg/src/arrow/reader.rs#L229-L235. + /// See: . fn predicate_pushdown( &self, predicate: Condition, From 1ba69ead81a17c465da2f2793c8e409131b07704 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 16:21:12 +0800 Subject: [PATCH 36/42] increase timeout --- ci/workflows/pull-request.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 3b315a7e5c8ff..e10ffb2d0091f 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -255,7 +255,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 17 + timeout_in_minutes: 15 retry: *auto-retry - label: "end-to-end iceberg sink v2 test" @@ -270,7 +270,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 15 + timeout_in_minutes: 17 retry: *auto-retry - label: "end-to-end iceberg cdc test" From a8601367992b215807ccb76bdf643f3f2aecfa79 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 19:38:48 +0800 Subject: [PATCH 37/42] use rule based predicate push down --- src/frontend/src/optimizer/mod.rs | 8 + .../src/optimizer/plan_node/batch_filter.rs | 6 + .../optimizer/plan_node/batch_iceberg_scan.rs | 12 +- .../plan_node/logical_iceberg_scan.rs | 295 +---------------- .../batch/batch_iceberg_predicate_pushdown.rs | 301 ++++++++++++++++++ src/frontend/src/optimizer/rule/batch/mod.rs | 1 + src/frontend/src/optimizer/rule/mod.rs | 2 + 7 files changed, 333 insertions(+), 292 deletions(-) create mode 100644 src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index e08f6b2c4dd4a..30d51bb93326c 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -402,6 +402,14 @@ impl PlanRoot { ApplyOrder::BottomUp, )); + // For iceberg scan, we do iceberg predicate pushdown + // BatchFilter -> BatchIcebergScan + let plan = plan.optimize_by_rules(&OptimizationStage::new( + "Iceberg Predicate Pushdown", + vec![BatchIcebergPredicatePushDownRule::create()], + ApplyOrder::BottomUp, + )); + assert_eq!(plan.convention(), Convention::Batch); Ok(plan) } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index ff89eacd485c0..6404fd852e6d6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -45,6 +45,12 @@ impl BatchFilter { pub fn predicate(&self) -> &Condition { &self.core.predicate } + + pub fn clone_with_predicate(&self, predicate: Condition) -> Self { + let mut core = self.core.clone(); + core.predicate = predicate; + Self::new(core) + } } impl_distill_by_unit!(BatchFilter, core, "BatchFilter"); diff --git a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs index 1c91d19a9fcc9..00f81df2b943c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs @@ -41,7 +41,7 @@ pub struct BatchIcebergScan { } impl BatchIcebergScan { - pub fn new(core: generic::Source, predicate: IcebergPredicate) -> Self { + pub fn new(core: generic::Source) -> Self { let base = PlanBase::new_batch_with_core( &core, // Use `Single` by default, will be updated later with `clone_with_dist`. @@ -52,7 +52,7 @@ impl BatchIcebergScan { Self { base, core, - predicate, + predicate: IcebergPredicate::AlwaysTrue, } } @@ -75,6 +75,14 @@ impl BatchIcebergScan { } } + pub fn clone_with_predicate(&self, predicate: IcebergPredicate) -> Self { + Self { + base: self.base.clone(), + core: self.core.clone(), + predicate, + } + } + pub fn as_of(&self) -> Option { self.core.as_of.clone() } diff --git a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs index 84a3217245266..b355d6f2057c8 100644 --- a/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs @@ -14,13 +14,7 @@ use std::rc::Rc; -use chrono::Datelike; -use educe::Educe; -use iceberg::expr::{Predicate as IcebergPredicate, Reference}; -use iceberg::spec::Datum as IcebergDatum; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::Field; -use risingwave_common::types::{Decimal, ScalarImpl}; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; @@ -30,7 +24,6 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; -use crate::expr::{ExprImpl, ExprType, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ @@ -40,14 +33,10 @@ use crate::optimizer::plan_node::{ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalIcebergScan` is only used by batch queries. At the beginning of the batch query optimization, `LogicalSource` with a iceberg property would be converted into a `LogicalIcebergScan`. -#[derive(Educe, Debug, Clone, PartialEq)] -#[educe(Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalIcebergScan { pub base: PlanBase, pub core: generic::Source, - /// TODO(kwannoel): If we support plan sharing, we can't just ignore the predicate. - #[educe(Hash(ignore))] - pub predicate: IcebergPredicate, } impl LogicalIcebergScan { @@ -59,11 +48,7 @@ impl LogicalIcebergScan { assert!(logical_source.output_exprs.is_none()); - LogicalIcebergScan { - base, - core, - predicate: IcebergPredicate::AlwaysTrue, - } + LogicalIcebergScan { base, core } } pub fn source_catalog(&self) -> Option> { @@ -79,21 +64,7 @@ impl LogicalIcebergScan { .collect(); let base = PlanBase::new_logical_with_core(&core); - LogicalIcebergScan { - base, - core, - predicate: self.predicate.clone(), - } - } - - pub fn clone_with_predicate(&self, predicate: IcebergPredicate) -> Self { - let base = PlanBase::new_logical_with_core(&self.core); - let predicate = predicate.and(self.predicate.clone()); - LogicalIcebergScan { - base, - core: self.core.clone(), - predicate, - } + LogicalIcebergScan { base, core } } } @@ -105,7 +76,6 @@ impl Distill for LogicalIcebergScan { vec![ ("source", src), ("columns", column_names_pretty(self.schema())), - ("predicate", Pretty::from(self.predicate.to_string())), ] } else { vec![] @@ -132,274 +102,19 @@ impl ExprRewritable for LogicalIcebergScan {} impl ExprVisitable for LogicalIcebergScan {} impl PredicatePushdown for LogicalIcebergScan { - /// NOTE(kwannoel): We do predicate pushdown to the iceberg-sdk here. - /// Currently we depend on third-party tools to compact iceberg tables and provide zone-map. - /// zone-map is used to evaluate predicates on iceberg tables. - /// Without zone-map, iceberg-sdk will still apply the predicate on its own. - /// See: . fn predicate_pushdown( &self, predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { - fn rw_literal_to_iceberg_datum(literal: &Literal) -> Option { - let Some(scalar) = literal.get_data() else { - return None; - }; - match scalar { - ScalarImpl::Bool(b) => Some(IcebergDatum::bool(*b)), - ScalarImpl::Int32(i) => Some(IcebergDatum::int(*i)), - ScalarImpl::Int64(i) => Some(IcebergDatum::long(*i)), - ScalarImpl::Float32(f) => Some(IcebergDatum::float(*f)), - ScalarImpl::Float64(f) => Some(IcebergDatum::double(*f)), - ScalarImpl::Decimal(d) => { - let Decimal::Normalized(d) = d else { - return None; - }; - let Ok(d) = IcebergDatum::decimal(*d) else { - return None; - }; - Some(d) - } - ScalarImpl::Date(d) => { - let Ok(datum) = IcebergDatum::date_from_ymd(d.0.year(), d.0.month(), d.0.day()) - else { - return None; - }; - Some(datum) - } - ScalarImpl::Timestamp(t) => Some(IcebergDatum::timestamp_nanos( - t.0.and_utc().timestamp_nanos_opt()?, - )), - ScalarImpl::Timestamptz(t) => { - Some(IcebergDatum::timestamptz_micros(t.timestamp_micros())) - } - ScalarImpl::Utf8(s) => Some(IcebergDatum::string(s)), - ScalarImpl::Bytea(b) => Some(IcebergDatum::binary(b.clone())), - _ => None, - } - } - - fn rw_expr_to_iceberg_predicate( - expr: &ExprImpl, - fields: &[Field], - ) -> Option { - match expr { - ExprImpl::Literal(l) => match l.get_data() { - Some(ScalarImpl::Bool(b)) => { - if *b { - Some(IcebergPredicate::AlwaysTrue) - } else { - Some(IcebergPredicate::AlwaysFalse) - } - } - _ => None, - }, - ExprImpl::FunctionCall(f) => { - let args = f.inputs(); - match f.func_type() { - ExprType::Not => { - let arg = rw_expr_to_iceberg_predicate(&args[0], fields)?; - Some(IcebergPredicate::negate(arg)) - } - ExprType::And => { - let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; - let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; - Some(IcebergPredicate::and(arg0, arg1)) - } - ExprType::Or => { - let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; - let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; - Some(IcebergPredicate::or(arg0, arg1)) - } - ExprType::Equal => match [&args[0], &args[1]] { - [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] - | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.equal_to(datum)) - } - _ => None, - }, - ExprType::NotEqual => match [&args[0], &args[1]] { - [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] - | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.not_equal_to(datum)) - } - _ => None, - }, - ExprType::GreaterThan => match [&args[0], &args[1]] { - [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than(datum)) - } - [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than_or_equal_to(datum)) - } - _ => None, - }, - ExprType::GreaterThanOrEqual => match [&args[0], &args[1]] { - [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than_or_equal_to(datum)) - } - [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than(datum)) - } - _ => None, - }, - ExprType::LessThan => match [&args[0], &args[1]] { - [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than(datum)) - } - [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than_or_equal_to(datum)) - } - _ => None, - }, - ExprType::LessThanOrEqual => match [&args[0], &args[1]] { - [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.less_than_or_equal_to(datum)) - } - [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let datum = rw_literal_to_iceberg_datum(rhs)?; - Some(reference.greater_than(datum)) - } - _ => None, - }, - ExprType::IsNull => match &args[0] { - ExprImpl::InputRef(lhs) => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - Some(reference.is_null()) - } - _ => None, - }, - ExprType::IsNotNull => match &args[0] { - ExprImpl::InputRef(lhs) => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - Some(reference.is_not_null()) - } - _ => None, - }, - ExprType::In => match &args[0] { - ExprImpl::InputRef(lhs) => { - let column_name = &fields[lhs.index].name; - let reference = Reference::new(column_name); - let mut datums = Vec::with_capacity(args.len() - 1); - for arg in &args[1..] { - if let ExprImpl::Literal(l) = arg { - if let Some(datum) = rw_literal_to_iceberg_datum(l) { - datums.push(datum); - } else { - return None; - } - } else { - return None; - } - } - Some(reference.is_in(datums)) - } - _ => None, - }, - _ => None, - } - } - _ => None, - } - } - fn rw_predicate_to_iceberg_predicate( - predicate: Condition, - fields: &[Field], - ) -> (IcebergPredicate, Condition) { - if predicate.always_true() { - return (IcebergPredicate::AlwaysTrue, predicate); - } - - let mut conjunctions = predicate.conjunctions; - let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); - - let mut iceberg_condition_root = None; - while let Some(conjunction) = conjunctions.pop() { - match rw_expr_to_iceberg_predicate(&conjunction, fields) { - iceberg_predicate @ Some(_) => { - iceberg_condition_root = iceberg_predicate; - break; - } - None => { - ignored_conjunctions.push(conjunction); - continue; - } - } - } - - let mut iceberg_condition_root = match iceberg_condition_root { - Some(p) => p, - None => { - return ( - IcebergPredicate::AlwaysTrue, - Condition { - conjunctions: ignored_conjunctions, - }, - ) - } - }; - - for rw_condition in conjunctions { - match rw_expr_to_iceberg_predicate(&rw_condition, fields) { - Some(iceberg_predicate) => { - iceberg_condition_root = iceberg_condition_root.and(iceberg_predicate) - } - None => ignored_conjunctions.push(rw_condition), - } - } - ( - iceberg_condition_root, - Condition { - conjunctions: ignored_conjunctions, - }, - ) - } - - let schema = self.schema(); - let fields = &schema.fields; - let (iceberg_predicate, rw_predicate) = - rw_predicate_to_iceberg_predicate(predicate, fields); // No pushdown. - let this = self.clone_with_predicate(iceberg_predicate); - LogicalFilter::create(this.into(), rw_predicate) + LogicalFilter::create(self.clone().into(), predicate) } } impl ToBatch for LogicalIcebergScan { fn to_batch(&self) -> Result { - let plan: PlanRef = BatchIcebergScan::new(self.core.clone(), self.predicate.clone()).into(); + let plan: PlanRef = BatchIcebergScan::new(self.core.clone()).into(); Ok(plan) } } diff --git a/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs b/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs new file mode 100644 index 0000000000000..cc6a23ca3936c --- /dev/null +++ b/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs @@ -0,0 +1,301 @@ +// 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. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +use chrono::Datelike; +use iceberg::expr::{Predicate as IcebergPredicate, Reference}; +use iceberg::spec::Datum as IcebergDatum; +use risingwave_common::catalog::Field; +use risingwave_common::types::{Decimal, ScalarImpl}; + +use crate::expr::{ExprImpl, ExprType, Literal}; +use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::{BatchFilter, BatchIcebergScan, PlanTreeNodeUnary}; +use crate::optimizer::rule::{BoxedRule, Rule}; +use crate::optimizer::PlanRef; +use crate::utils::Condition; + +/// NOTE(kwannoel): We do predicate pushdown to the iceberg-sdk here. +/// zone-map is used to evaluate predicates on iceberg tables. +/// Without zone-map, iceberg-sdk will still apply the predicate on its own. +/// See: . +pub struct BatchIcebergPredicatePushDownRule {} + +impl Rule for BatchIcebergPredicatePushDownRule { + fn apply(&self, plan: PlanRef) -> Option { + let filter: &BatchFilter = plan.as_batch_filter()?; + let input = filter.input(); + let scan: &BatchIcebergScan = input.as_batch_iceberg_scan()?; + // NOTE(kwannoel): We only fill iceberg predicate here. + assert_eq!(scan.predicate, IcebergPredicate::AlwaysTrue); + + let predicate = filter.predicate().clone(); + let (iceberg_predicate, new_predicate) = + rw_predicate_to_iceberg_predicate(predicate, scan.schema().fields()); + let scan = scan.clone_with_predicate(iceberg_predicate); + let filter = filter + .clone_with_input(scan.into()) + .clone_with_predicate(new_predicate); + Some(filter.into()) + } +} + +fn rw_literal_to_iceberg_datum(literal: &Literal) -> Option { + let Some(scalar) = literal.get_data() else { + return None; + }; + match scalar { + ScalarImpl::Bool(b) => Some(IcebergDatum::bool(*b)), + ScalarImpl::Int32(i) => Some(IcebergDatum::int(*i)), + ScalarImpl::Int64(i) => Some(IcebergDatum::long(*i)), + ScalarImpl::Float32(f) => Some(IcebergDatum::float(*f)), + ScalarImpl::Float64(f) => Some(IcebergDatum::double(*f)), + ScalarImpl::Decimal(d) => { + let Decimal::Normalized(d) = d else { + return None; + }; + let Ok(d) = IcebergDatum::decimal(*d) else { + return None; + }; + Some(d) + } + ScalarImpl::Date(d) => { + let Ok(datum) = IcebergDatum::date_from_ymd(d.0.year(), d.0.month(), d.0.day()) else { + return None; + }; + Some(datum) + } + ScalarImpl::Timestamp(t) => Some(IcebergDatum::timestamp_nanos( + t.0.and_utc().timestamp_nanos_opt()?, + )), + ScalarImpl::Timestamptz(t) => Some(IcebergDatum::timestamptz_micros(t.timestamp_micros())), + ScalarImpl::Utf8(s) => Some(IcebergDatum::string(s)), + ScalarImpl::Bytea(b) => Some(IcebergDatum::binary(b.clone())), + _ => None, + } +} + +fn rw_expr_to_iceberg_predicate(expr: &ExprImpl, fields: &[Field]) -> Option { + match expr { + ExprImpl::Literal(l) => match l.get_data() { + Some(ScalarImpl::Bool(b)) => { + if *b { + Some(IcebergPredicate::AlwaysTrue) + } else { + Some(IcebergPredicate::AlwaysFalse) + } + } + _ => None, + }, + ExprImpl::FunctionCall(f) => { + let args = f.inputs(); + match f.func_type() { + ExprType::Not => { + let arg = rw_expr_to_iceberg_predicate(&args[0], fields)?; + Some(IcebergPredicate::negate(arg)) + } + ExprType::And => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; + Some(IcebergPredicate::and(arg0, arg1)) + } + ExprType::Or => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; + Some(IcebergPredicate::or(arg0, arg1)) + } + ExprType::Equal => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] + | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.equal_to(datum)) + } + _ => None, + }, + ExprType::NotEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] + | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.not_equal_to(datum)) + } + _ => None, + }, + ExprType::GreaterThan => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) + } + _ => None, + }, + ExprType::GreaterThanOrEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) + } + _ => None, + }, + ExprType::LessThan => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) + } + _ => None, + }, + ExprType::LessThanOrEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) + } + _ => None, + }, + ExprType::IsNull => match &args[0] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_null()) + } + _ => None, + }, + ExprType::IsNotNull => match &args[0] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_not_null()) + } + _ => None, + }, + ExprType::In => match &args[0] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let mut datums = Vec::with_capacity(args.len() - 1); + for arg in &args[1..] { + if let ExprImpl::Literal(l) = arg { + if let Some(datum) = rw_literal_to_iceberg_datum(l) { + datums.push(datum); + } else { + return None; + } + } else { + return None; + } + } + Some(reference.is_in(datums)) + } + _ => None, + }, + _ => None, + } + } + _ => None, + } +} +fn rw_predicate_to_iceberg_predicate( + predicate: Condition, + fields: &[Field], +) -> (IcebergPredicate, Condition) { + if predicate.always_true() { + return (IcebergPredicate::AlwaysTrue, predicate); + } + + let mut conjunctions = predicate.conjunctions; + let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); + + let mut iceberg_condition_root = None; + while let Some(conjunction) = conjunctions.pop() { + match rw_expr_to_iceberg_predicate(&conjunction, fields) { + iceberg_predicate @ Some(_) => { + iceberg_condition_root = iceberg_predicate; + break; + } + None => { + ignored_conjunctions.push(conjunction); + continue; + } + } + } + + let mut iceberg_condition_root = match iceberg_condition_root { + Some(p) => p, + None => { + return ( + IcebergPredicate::AlwaysTrue, + Condition { + conjunctions: ignored_conjunctions, + }, + ) + } + }; + + for rw_condition in conjunctions { + match rw_expr_to_iceberg_predicate(&rw_condition, fields) { + Some(iceberg_predicate) => { + iceberg_condition_root = iceberg_condition_root.and(iceberg_predicate) + } + None => ignored_conjunctions.push(rw_condition), + } + } + ( + iceberg_condition_root, + Condition { + conjunctions: ignored_conjunctions, + }, + ) +} + +impl BatchIcebergPredicatePushDownRule { + pub fn create() -> BoxedRule { + Box::new(BatchIcebergPredicatePushDownRule {}) + } +} diff --git a/src/frontend/src/optimizer/rule/batch/mod.rs b/src/frontend/src/optimizer/rule/batch/mod.rs index 6061c985b6696..c4d31faf3cfba 100644 --- a/src/frontend/src/optimizer/rule/batch/mod.rs +++ b/src/frontend/src/optimizer/rule/batch/mod.rs @@ -12,5 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod batch_iceberg_predicate_pushdown; pub(crate) mod batch_project_merge_rule; pub mod batch_push_limit_to_scan_rule; diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index 7468f1c96524c..e9bd08e6c6797 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -165,6 +165,7 @@ mod table_function_to_mysql_query_rule; mod table_function_to_postgres_query_rule; mod values_extract_project_rule; +pub use batch::batch_iceberg_predicate_pushdown::*; pub use batch::batch_push_limit_to_scan_rule::*; pub use pull_up_correlated_predicate_agg_rule::*; pub use source_to_iceberg_scan_rule::*; @@ -248,6 +249,7 @@ macro_rules! for_all_rules { , { AggCallMergeRule } , { ValuesExtractProjectRule } , { BatchPushLimitToScanRule } + , { BatchIcebergPredicatePushDownRule } , { PullUpCorrelatedPredicateAggRule } , { SourceToKafkaScanRule } , { SourceToIcebergScanRule } From 69cad02d3b0f83ee0fce5e17ac6724ee6088187d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 22:17:50 +0800 Subject: [PATCH 38/42] prune BatchFilter if predicate always true --- .../rule/batch/batch_iceberg_predicate_pushdown.rs | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs b/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs index cc6a23ca3936c..8df8777d59384 100644 --- a/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs +++ b/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs @@ -45,13 +45,17 @@ impl Rule for BatchIcebergPredicatePushDownRule { assert_eq!(scan.predicate, IcebergPredicate::AlwaysTrue); let predicate = filter.predicate().clone(); - let (iceberg_predicate, new_predicate) = + let (iceberg_predicate, rw_predicate) = rw_predicate_to_iceberg_predicate(predicate, scan.schema().fields()); let scan = scan.clone_with_predicate(iceberg_predicate); - let filter = filter - .clone_with_input(scan.into()) - .clone_with_predicate(new_predicate); - Some(filter.into()) + if rw_predicate.always_true() { + Some(scan.into()) + } else { + let filter = filter + .clone_with_input(scan.into()) + .clone_with_predicate(rw_predicate); + Some(filter.into()) + } } } From 311f3d0a837da6c8ae40b47bba73287c8e72d7bf Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 22:23:50 +0800 Subject: [PATCH 39/42] test mix filter and predicate --- .../iceberg/test_case/iceberg_predicate_pushdown.slt | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt index 4cee28f4e0ea6..cb0855c77593e 100644 --- a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -116,6 +116,18 @@ select count(*), i2, i3 from iceberg_t1_source where i2 = 'some str' and i3 = 'a ---- 500 some str another str +query I +explain select i1 from iceberg_t1_source where i1 > 500 and i2 = i3; +---- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [i1] } + └─BatchFilter { predicate: (i2 = i3) } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: i1 > 500 } + +query I +select i1 from iceberg_t1_source where i1 > 500 and i2 = i3; +---- + statement ok DROP SINK sink1; From 5df282d5b5e88ae0db8ad2e9be8577d9ebfbd090 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 14 Nov 2024 10:44:57 +0800 Subject: [PATCH 40/42] handle case where iceberg split is empty --- .../test_case/iceberg_predicate_pushdown.slt | 5 ++++ src/connector/src/source/iceberg/mod.rs | 24 ++++++++++++------- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt index cb0855c77593e..2075d129a8a1e 100644 --- a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -128,6 +128,11 @@ query I select i1 from iceberg_t1_source where i1 > 500 and i2 = i3; ---- +# Empty splits should not panic +query I +select i1 from iceberg_t1_source where i1 > 1001; +---- + statement ok DROP SINK sink1; diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 5c879dea63f12..914479a672b39 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -138,6 +138,19 @@ pub struct IcebergSplit { pub position_delete_files: Vec, } +impl IcebergSplit { + pub fn empty() -> Self { + Self { + split_id: 0, + snapshot_id: 0, + table_meta: TableMetadataJsonStr("".to_string()), + files: vec![], + equality_delete_files: vec![], + position_delete_files: vec![], + } + } +} + impl SplitMetaData for IcebergSplit { fn id(&self) -> SplitId { self.split_id.to_string().into() @@ -201,14 +214,7 @@ impl IcebergSplitEnumerator { let current_snapshot = table.metadata().current_snapshot(); if current_snapshot.is_none() { // If there is no snapshot, we will return a mock `IcebergSplit` with empty files. - return Ok(vec![IcebergSplit { - split_id: 0, - snapshot_id: 0, - table_meta: TableMetadataJsonStr::serialize(table.metadata()), - files: vec![], - equality_delete_files: vec![], - position_delete_files: vec![], - }]); + return Ok(vec![IcebergSplit::empty()]); } let snapshot_id = match time_traval_info { @@ -314,7 +320,7 @@ impl IcebergSplitEnumerator { .collect_vec(); if splits.is_empty() { - bail!("No splits found for the iceberg table"); + return Ok(vec![IcebergSplit::empty()]); } Ok(splits) From b4187d3220d10c0cead7da9077a81c8c799cb47e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 14 Nov 2024 10:56:32 +0800 Subject: [PATCH 41/42] panic if doing unsupported ops --- .../optimizer/plan_node/batch_iceberg_scan.rs | 31 ++++++++++++++++--- 1 file changed, 27 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs index 00f81df2b943c..815b711faa296 100644 --- a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::hash::{Hash, Hasher}; use std::rc::Rc; -use educe::Educe; use iceberg::expr::Predicate as IcebergPredicate; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; @@ -31,15 +31,38 @@ use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::property::{Distribution, Order}; -#[derive(Educe, Debug, Clone, PartialEq)] -#[educe(Eq, Hash)] +#[derive(Debug, Clone)] pub struct BatchIcebergScan { pub base: PlanBase, pub core: generic::Source, - #[educe(Hash(ignore))] pub predicate: IcebergPredicate, } +impl PartialEq for BatchIcebergScan { + fn eq(&self, other: &Self) -> bool { + if self.predicate == IcebergPredicate::AlwaysTrue + && other.predicate == IcebergPredicate::AlwaysTrue + { + self.base == other.base && self.core == other.core + } else { + panic!("BatchIcebergScan::eq: comparing non-AlwaysTrue predicates is not supported") + } + } +} + +impl Eq for BatchIcebergScan {} + +impl Hash for BatchIcebergScan { + fn hash(&self, state: &mut H) { + if self.predicate != IcebergPredicate::AlwaysTrue { + panic!("BatchIcebergScan::hash: hashing non-AlwaysTrue predicates is not supported") + } else { + self.base.hash(state); + self.core.hash(state); + } + } +} + impl BatchIcebergScan { pub fn new(core: generic::Source) -> Self { let base = PlanBase::new_batch_with_core( From f98b5febf7e713e8d6308a4d61384f2a409fe478 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 14 Nov 2024 12:32:08 +0800 Subject: [PATCH 42/42] fix metadata --- src/connector/src/source/iceberg/mod.rs | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 914479a672b39..aeb642c80a014 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -139,11 +139,11 @@ pub struct IcebergSplit { } impl IcebergSplit { - pub fn empty() -> Self { + pub fn empty(table_meta: TableMetadataJsonStr) -> Self { Self { split_id: 0, snapshot_id: 0, - table_meta: TableMetadataJsonStr("".to_string()), + table_meta, files: vec![], equality_delete_files: vec![], position_delete_files: vec![], @@ -214,7 +214,9 @@ impl IcebergSplitEnumerator { let current_snapshot = table.metadata().current_snapshot(); if current_snapshot.is_none() { // If there is no snapshot, we will return a mock `IcebergSplit` with empty files. - return Ok(vec![IcebergSplit::empty()]); + return Ok(vec![IcebergSplit::empty(TableMetadataJsonStr::serialize( + table.metadata(), + ))]); } let snapshot_id = match time_traval_info { @@ -320,7 +322,9 @@ impl IcebergSplitEnumerator { .collect_vec(); if splits.is_empty() { - return Ok(vec![IcebergSplit::empty()]); + return Ok(vec![IcebergSplit::empty(TableMetadataJsonStr::serialize( + table.metadata(), + ))]); } Ok(splits)