From bbfab6b6405b012c643ee03f6b767864ab570522 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Fri, 13 Dec 2024 11:32:41 -0600 Subject: [PATCH 01/23] signature helper method --- crates/rayexec_execution/src/functions/mod.rs | 10 +- .../src/functions/scalar/builtin/arith/add.rs | 113 ++++++--------- .../src/functions/scalar/builtin/arith/div.rs | 122 ++++++---------- .../src/functions/scalar/builtin/arith/mul.rs | 122 ++++++---------- .../src/functions/scalar/builtin/arith/rem.rs | 108 ++++++-------- .../src/functions/scalar/builtin/arith/sub.rs | 132 +++++++----------- .../src/functions/scalar/builtin/negate.rs | 59 ++------ 7 files changed, 243 insertions(+), 423 deletions(-) diff --git a/crates/rayexec_execution/src/functions/mod.rs b/crates/rayexec_execution/src/functions/mod.rs index 5401e7efd..3bd423197 100644 --- a/crates/rayexec_execution/src/functions/mod.rs +++ b/crates/rayexec_execution/src/functions/mod.rs @@ -16,7 +16,7 @@ use rayexec_error::{RayexecError, Result}; /// Function signature. #[derive(Debug, Clone, PartialEq)] pub struct Signature { - /// Expected input types for this signature. + /// Expected positional input types for this signature. pub input: &'static [DataTypeId], /// Type of the variadic args if this function is variadic. @@ -39,6 +39,14 @@ pub struct Signature { } impl Signature { + pub const fn new_positional(input: &'static [DataTypeId], return_type: DataTypeId) -> Self { + Signature { + input, + variadic: None, + return_type, + } + } + /// Check if this signature is a variadic signature. pub const fn is_variadic(&self) -> bool { self.variadic.is_some() diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/arith/add.rs b/crates/rayexec_execution/src/functions/scalar/builtin/arith/add.rs index ef42275c2..c573d5943 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/arith/add.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/arith/add.rs @@ -42,78 +42,47 @@ impl FunctionInfo for Add { } fn signatures(&self) -> &[Signature] { - &[ - Signature { - input: &[DataTypeId::Float16, DataTypeId::Float16], - variadic: None, - return_type: DataTypeId::Float16, - }, - Signature { - input: &[DataTypeId::Float32, DataTypeId::Float32], - variadic: None, - return_type: DataTypeId::Float32, - }, - Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, - return_type: DataTypeId::Float64, - }, - Signature { - input: &[DataTypeId::Int8, DataTypeId::Int8], - variadic: None, - return_type: DataTypeId::Int8, - }, - Signature { - input: &[DataTypeId::Int16, DataTypeId::Int16], - variadic: None, - return_type: DataTypeId::Int16, - }, - Signature { - input: &[DataTypeId::Int32, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Int32, - }, - Signature { - input: &[DataTypeId::Int64, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Int64, - }, - Signature { - input: &[DataTypeId::UInt8, DataTypeId::UInt8], - variadic: None, - return_type: DataTypeId::UInt8, - }, - Signature { - input: &[DataTypeId::UInt16, DataTypeId::UInt16], - variadic: None, - return_type: DataTypeId::UInt16, - }, - Signature { - input: &[DataTypeId::UInt32, DataTypeId::UInt32], - variadic: None, - return_type: DataTypeId::UInt32, - }, - Signature { - input: &[DataTypeId::UInt64, DataTypeId::UInt64], - variadic: None, - return_type: DataTypeId::UInt64, - }, - Signature { - input: &[DataTypeId::Date32, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Date32, - }, - Signature { - input: &[DataTypeId::Interval, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Interval, - }, - Signature { - input: &[DataTypeId::Decimal64, DataTypeId::Decimal64], - variadic: None, - return_type: DataTypeId::Decimal64, - }, - ] + const SIGS: &[Signature] = &[ + Signature::new_positional( + &[DataTypeId::Float16, DataTypeId::Float16], + DataTypeId::Float16, + ), + Signature::new_positional( + &[DataTypeId::Float32, DataTypeId::Float32], + DataTypeId::Float32, + ), + Signature::new_positional( + &[DataTypeId::Float64, DataTypeId::Float64], + DataTypeId::Float64, + ), + Signature::new_positional(&[DataTypeId::Int8, DataTypeId::Int8], DataTypeId::Int8), + Signature::new_positional(&[DataTypeId::Int16, DataTypeId::Int16], DataTypeId::Int16), + Signature::new_positional(&[DataTypeId::Int32, DataTypeId::Int32], DataTypeId::Int32), + Signature::new_positional(&[DataTypeId::Int64, DataTypeId::Int64], DataTypeId::Int64), + Signature::new_positional(&[DataTypeId::UInt8, DataTypeId::UInt8], DataTypeId::UInt8), + Signature::new_positional( + &[DataTypeId::UInt16, DataTypeId::UInt16], + DataTypeId::UInt16, + ), + Signature::new_positional( + &[DataTypeId::UInt32, DataTypeId::UInt32], + DataTypeId::UInt32, + ), + Signature::new_positional( + &[DataTypeId::UInt64, DataTypeId::UInt64], + DataTypeId::UInt64, + ), + Signature::new_positional(&[DataTypeId::Date32, DataTypeId::Int32], DataTypeId::Date32), + Signature::new_positional( + &[DataTypeId::Interval, DataTypeId::Int64], + DataTypeId::Interval, + ), + Signature::new_positional( + &[DataTypeId::Decimal64, DataTypeId::Decimal64], + DataTypeId::Decimal64, + ), + ]; + SIGS } } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/arith/div.rs b/crates/rayexec_execution/src/functions/scalar/builtin/arith/div.rs index 9b65258a5..242834269 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/arith/div.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/arith/div.rs @@ -45,83 +45,51 @@ impl FunctionInfo for Div { } fn signatures(&self) -> &[Signature] { - &[ - Signature { - input: &[DataTypeId::Float16, DataTypeId::Float16], - variadic: None, - return_type: DataTypeId::Float16, - }, - Signature { - input: &[DataTypeId::Float32, DataTypeId::Float32], - variadic: None, - return_type: DataTypeId::Float32, - }, - Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, - return_type: DataTypeId::Float64, - }, - Signature { - input: &[DataTypeId::Int8, DataTypeId::Int8], - variadic: None, - return_type: DataTypeId::Int8, - }, - Signature { - input: &[DataTypeId::Int16, DataTypeId::Int16], - variadic: None, - return_type: DataTypeId::Int16, - }, - Signature { - input: &[DataTypeId::Int32, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Int32, - }, - Signature { - input: &[DataTypeId::Int64, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Int64, - }, - Signature { - input: &[DataTypeId::UInt8, DataTypeId::UInt8], - variadic: None, - return_type: DataTypeId::UInt8, - }, - Signature { - input: &[DataTypeId::UInt16, DataTypeId::UInt16], - variadic: None, - return_type: DataTypeId::UInt16, - }, - Signature { - input: &[DataTypeId::UInt32, DataTypeId::UInt32], - variadic: None, - return_type: DataTypeId::UInt32, - }, - Signature { - input: &[DataTypeId::UInt64, DataTypeId::UInt64], - variadic: None, - return_type: DataTypeId::UInt64, - }, - Signature { - input: &[DataTypeId::Date32, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Date32, - }, - Signature { - input: &[DataTypeId::Interval, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Interval, - }, - Signature { - input: &[DataTypeId::Decimal64, DataTypeId::Decimal64], - variadic: None, - return_type: DataTypeId::Float64, - }, - Signature { - input: &[DataTypeId::Decimal128, DataTypeId::Decimal128], - variadic: None, - return_type: DataTypeId::Float64, - }, - ] + const SIGS: &[Signature] = &[ + Signature::new_positional( + &[DataTypeId::Float16, DataTypeId::Float16], + DataTypeId::Float16, + ), + Signature::new_positional( + &[DataTypeId::Float32, DataTypeId::Float32], + DataTypeId::Float32, + ), + Signature::new_positional( + &[DataTypeId::Float64, DataTypeId::Float64], + DataTypeId::Float64, + ), + Signature::new_positional(&[DataTypeId::Int8, DataTypeId::Int8], DataTypeId::Int8), + Signature::new_positional(&[DataTypeId::Int16, DataTypeId::Int16], DataTypeId::Int16), + Signature::new_positional(&[DataTypeId::Int32, DataTypeId::Int32], DataTypeId::Int32), + Signature::new_positional(&[DataTypeId::Int64, DataTypeId::Int64], DataTypeId::Int64), + Signature::new_positional(&[DataTypeId::UInt8, DataTypeId::UInt8], DataTypeId::UInt8), + Signature::new_positional( + &[DataTypeId::UInt16, DataTypeId::UInt16], + DataTypeId::UInt16, + ), + Signature::new_positional( + &[DataTypeId::UInt32, DataTypeId::UInt32], + DataTypeId::UInt32, + ), + Signature::new_positional( + &[DataTypeId::UInt64, DataTypeId::UInt64], + DataTypeId::UInt64, + ), + Signature::new_positional(&[DataTypeId::Date32, DataTypeId::Int64], DataTypeId::Date32), + Signature::new_positional( + &[DataTypeId::Interval, DataTypeId::Int64], + DataTypeId::Interval, + ), + Signature::new_positional( + &[DataTypeId::Decimal64, DataTypeId::Decimal64], + DataTypeId::Float64, + ), + Signature::new_positional( + &[DataTypeId::Decimal128, DataTypeId::Decimal128], + DataTypeId::Float64, + ), + ]; + SIGS } } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/arith/mul.rs b/crates/rayexec_execution/src/functions/scalar/builtin/arith/mul.rs index 3776e986d..04f236827 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/arith/mul.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/arith/mul.rs @@ -46,83 +46,51 @@ impl FunctionInfo for Mul { } fn signatures(&self) -> &[Signature] { - &[ - Signature { - input: &[DataTypeId::Float16, DataTypeId::Float16], - variadic: None, - return_type: DataTypeId::Float16, - }, - Signature { - input: &[DataTypeId::Float32, DataTypeId::Float32], - variadic: None, - return_type: DataTypeId::Float32, - }, - Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, - return_type: DataTypeId::Float64, - }, - Signature { - input: &[DataTypeId::Int8, DataTypeId::Int8], - variadic: None, - return_type: DataTypeId::Int8, - }, - Signature { - input: &[DataTypeId::Int16, DataTypeId::Int16], - variadic: None, - return_type: DataTypeId::Int16, - }, - Signature { - input: &[DataTypeId::Int32, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Int32, - }, - Signature { - input: &[DataTypeId::Int64, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Int64, - }, - Signature { - input: &[DataTypeId::UInt8, DataTypeId::UInt8], - variadic: None, - return_type: DataTypeId::UInt8, - }, - Signature { - input: &[DataTypeId::UInt16, DataTypeId::UInt16], - variadic: None, - return_type: DataTypeId::UInt16, - }, - Signature { - input: &[DataTypeId::UInt32, DataTypeId::UInt32], - variadic: None, - return_type: DataTypeId::UInt32, - }, - Signature { - input: &[DataTypeId::UInt64, DataTypeId::UInt64], - variadic: None, - return_type: DataTypeId::UInt64, - }, - Signature { - input: &[DataTypeId::Date32, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Date32, - }, - Signature { - input: &[DataTypeId::Interval, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Interval, - }, - Signature { - input: &[DataTypeId::Interval, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Interval, - }, - Signature { - input: &[DataTypeId::Decimal64, DataTypeId::Decimal64], - variadic: None, - return_type: DataTypeId::Decimal64, - }, - ] + const SIGS: &[Signature] = &[ + Signature::new_positional( + &[DataTypeId::Float16, DataTypeId::Float16], + DataTypeId::Float16, + ), + Signature::new_positional( + &[DataTypeId::Float32, DataTypeId::Float32], + DataTypeId::Float32, + ), + Signature::new_positional( + &[DataTypeId::Float64, DataTypeId::Float64], + DataTypeId::Float64, + ), + Signature::new_positional(&[DataTypeId::Int8, DataTypeId::Int8], DataTypeId::Int8), + Signature::new_positional(&[DataTypeId::Int16, DataTypeId::Int16], DataTypeId::Int16), + Signature::new_positional(&[DataTypeId::Int32, DataTypeId::Int32], DataTypeId::Int32), + Signature::new_positional(&[DataTypeId::Int64, DataTypeId::Int64], DataTypeId::Int64), + Signature::new_positional(&[DataTypeId::UInt8, DataTypeId::UInt8], DataTypeId::UInt8), + Signature::new_positional( + &[DataTypeId::UInt16, DataTypeId::UInt16], + DataTypeId::UInt16, + ), + Signature::new_positional( + &[DataTypeId::UInt32, DataTypeId::UInt32], + DataTypeId::UInt32, + ), + Signature::new_positional( + &[DataTypeId::UInt64, DataTypeId::UInt64], + DataTypeId::UInt64, + ), + Signature::new_positional(&[DataTypeId::Date32, DataTypeId::Int64], DataTypeId::Date32), + Signature::new_positional( + &[DataTypeId::Interval, DataTypeId::Int32], + DataTypeId::Interval, + ), + Signature::new_positional( + &[DataTypeId::Interval, DataTypeId::Int64], + DataTypeId::Interval, + ), + Signature::new_positional( + &[DataTypeId::Decimal64, DataTypeId::Decimal64], + DataTypeId::Decimal64, + ), + ]; + SIGS } } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/arith/rem.rs b/crates/rayexec_execution/src/functions/scalar/builtin/arith/rem.rs index a4d865534..85ab0d93d 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/arith/rem.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/arith/rem.rs @@ -42,72 +42,45 @@ impl FunctionInfo for Rem { } fn signatures(&self) -> &[Signature] { - &[ - Signature { - input: &[DataTypeId::Float16, DataTypeId::Float16], - variadic: None, - return_type: DataTypeId::Float16, - }, - Signature { - input: &[DataTypeId::Float32, DataTypeId::Float32], - variadic: None, - return_type: DataTypeId::Float32, - }, - Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, - return_type: DataTypeId::Float64, - }, - Signature { - input: &[DataTypeId::Int8, DataTypeId::Int8], - variadic: None, - return_type: DataTypeId::Int8, - }, - Signature { - input: &[DataTypeId::Int16, DataTypeId::Int16], - variadic: None, - return_type: DataTypeId::Int16, - }, - Signature { - input: &[DataTypeId::Int32, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Int32, - }, - Signature { - input: &[DataTypeId::Int64, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Int64, - }, - Signature { - input: &[DataTypeId::Int128, DataTypeId::Int128], - variadic: None, - return_type: DataTypeId::Int128, - }, - Signature { - input: &[DataTypeId::UInt8, DataTypeId::UInt8], - variadic: None, - return_type: DataTypeId::UInt8, - }, - Signature { - input: &[DataTypeId::UInt16, DataTypeId::UInt16], - variadic: None, - return_type: DataTypeId::UInt16, - }, - Signature { - input: &[DataTypeId::UInt32, DataTypeId::UInt32], - variadic: None, - return_type: DataTypeId::UInt32, - }, - Signature { - input: &[DataTypeId::UInt64, DataTypeId::UInt64], - variadic: None, - return_type: DataTypeId::UInt64, - }, - Signature { - input: &[DataTypeId::UInt128, DataTypeId::UInt128], - variadic: None, - return_type: DataTypeId::UInt128, - }, + const SIGS: &[Signature] = &[ + Signature::new_positional( + &[DataTypeId::Float16, DataTypeId::Float16], + DataTypeId::Float16, + ), + Signature::new_positional( + &[DataTypeId::Float32, DataTypeId::Float32], + DataTypeId::Float32, + ), + Signature::new_positional( + &[DataTypeId::Float64, DataTypeId::Float64], + DataTypeId::Float64, + ), + Signature::new_positional(&[DataTypeId::Int8, DataTypeId::Int8], DataTypeId::Int8), + Signature::new_positional(&[DataTypeId::Int16, DataTypeId::Int16], DataTypeId::Int16), + Signature::new_positional(&[DataTypeId::Int32, DataTypeId::Int32], DataTypeId::Int32), + Signature::new_positional(&[DataTypeId::Int64, DataTypeId::Int64], DataTypeId::Int64), + Signature::new_positional( + &[DataTypeId::Int128, DataTypeId::Int128], + DataTypeId::Int128, + ), + Signature::new_positional(&[DataTypeId::UInt8, DataTypeId::UInt8], DataTypeId::UInt8), + Signature::new_positional( + &[DataTypeId::UInt16, DataTypeId::UInt16], + DataTypeId::UInt16, + ), + Signature::new_positional( + &[DataTypeId::UInt32, DataTypeId::UInt32], + DataTypeId::UInt32, + ), + Signature::new_positional( + &[DataTypeId::UInt64, DataTypeId::UInt64], + DataTypeId::UInt64, + ), + Signature::new_positional( + &[DataTypeId::UInt128, DataTypeId::UInt128], + DataTypeId::UInt128, + ), + // TODO // Signature { // input: &[DataTypeId::Date32, DataTypeId::Int64], // variadic: None, @@ -123,7 +96,8 @@ impl FunctionInfo for Rem { // variadic: None, // return_type: DataTypeId::Decimal64, // }, - ] + ]; + SIGS } } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/arith/sub.rs b/crates/rayexec_execution/src/functions/scalar/builtin/arith/sub.rs index ce414ec96..5ee9db3c6 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/arith/sub.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/arith/sub.rs @@ -42,93 +42,61 @@ impl FunctionInfo for Sub { } fn signatures(&self) -> &[Signature] { - &[ - Signature { - input: &[DataTypeId::Float16, DataTypeId::Float16], - variadic: None, - return_type: DataTypeId::Float16, - }, - Signature { - input: &[DataTypeId::Float32, DataTypeId::Float32], - variadic: None, - return_type: DataTypeId::Float32, - }, - Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, - return_type: DataTypeId::Float64, - }, - Signature { - input: &[DataTypeId::Int8, DataTypeId::Int8], - variadic: None, - return_type: DataTypeId::Int8, - }, - Signature { - input: &[DataTypeId::Int16, DataTypeId::Int16], - variadic: None, - return_type: DataTypeId::Int16, - }, - Signature { - input: &[DataTypeId::Int32, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Int32, - }, - Signature { - input: &[DataTypeId::Int64, DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Int64, - }, - Signature { - input: &[DataTypeId::Int128, DataTypeId::Int128], - variadic: None, - return_type: DataTypeId::Int128, - }, - Signature { - input: &[DataTypeId::UInt8, DataTypeId::UInt8], - variadic: None, - return_type: DataTypeId::UInt8, - }, - Signature { - input: &[DataTypeId::UInt16, DataTypeId::UInt16], - variadic: None, - return_type: DataTypeId::UInt16, - }, - Signature { - input: &[DataTypeId::UInt32, DataTypeId::UInt32], - variadic: None, - return_type: DataTypeId::UInt32, - }, - Signature { - input: &[DataTypeId::UInt64, DataTypeId::UInt64], - variadic: None, - return_type: DataTypeId::UInt64, - }, - Signature { - input: &[DataTypeId::UInt128, DataTypeId::UInt128], - variadic: None, - return_type: DataTypeId::UInt128, - }, - Signature { - input: &[DataTypeId::Date32, DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Date32, - }, + const SIGS: &[Signature] = &[ + Signature::new_positional( + &[DataTypeId::Float16, DataTypeId::Float16], + DataTypeId::Float16, + ), + Signature::new_positional( + &[DataTypeId::Float32, DataTypeId::Float32], + DataTypeId::Float32, + ), + Signature::new_positional( + &[DataTypeId::Float64, DataTypeId::Float64], + DataTypeId::Float64, + ), + Signature::new_positional(&[DataTypeId::Int8, DataTypeId::Int8], DataTypeId::Int8), + Signature::new_positional(&[DataTypeId::Int16, DataTypeId::Int16], DataTypeId::Int16), + Signature::new_positional(&[DataTypeId::Int32, DataTypeId::Int32], DataTypeId::Int32), + Signature::new_positional(&[DataTypeId::Int64, DataTypeId::Int64], DataTypeId::Int64), + Signature::new_positional( + &[DataTypeId::Int128, DataTypeId::Int128], + DataTypeId::Int128, + ), + Signature::new_positional(&[DataTypeId::UInt8, DataTypeId::UInt8], DataTypeId::UInt8), + Signature::new_positional( + &[DataTypeId::UInt16, DataTypeId::UInt16], + DataTypeId::UInt16, + ), + Signature::new_positional( + &[DataTypeId::UInt32, DataTypeId::UInt32], + DataTypeId::UInt32, + ), + Signature::new_positional( + &[DataTypeId::UInt64, DataTypeId::UInt64], + DataTypeId::UInt64, + ), + Signature::new_positional( + &[DataTypeId::UInt128, DataTypeId::UInt128], + DataTypeId::UInt128, + ), + Signature::new_positional(&[DataTypeId::Date32, DataTypeId::Int32], DataTypeId::Date32), + // TODO // Signature { // input: &[DataTypeId::Interval, DataTypeId::Int64], // variadic: None, // return_type: DataTypeId::Interval, // }, - Signature { - input: &[DataTypeId::Decimal64, DataTypeId::Decimal64], - variadic: None, - return_type: DataTypeId::Decimal64, - }, - Signature { - input: &[DataTypeId::Decimal128, DataTypeId::Decimal128], - variadic: None, - return_type: DataTypeId::Decimal128, - }, - ] + Signature::new_positional( + &[DataTypeId::Decimal64, DataTypeId::Decimal64], + DataTypeId::Decimal64, + ), + Signature::new_positional( + &[DataTypeId::Decimal128, DataTypeId::Decimal128], + DataTypeId::Decimal128, + ), + ]; + SIGS } } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs b/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs index 144b3c3e7..0095bb341 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs @@ -33,53 +33,18 @@ impl FunctionInfo for Negate { } fn signatures(&self) -> &[Signature] { - &[ - Signature { - input: &[DataTypeId::Float16], - variadic: None, - return_type: DataTypeId::Float16, - }, - Signature { - input: &[DataTypeId::Float32], - variadic: None, - return_type: DataTypeId::Float32, - }, - Signature { - input: &[DataTypeId::Float64], - variadic: None, - return_type: DataTypeId::Float64, - }, - Signature { - input: &[DataTypeId::Int8], - variadic: None, - return_type: DataTypeId::Int8, - }, - Signature { - input: &[DataTypeId::Int16], - variadic: None, - return_type: DataTypeId::Int16, - }, - Signature { - input: &[DataTypeId::Int32], - variadic: None, - return_type: DataTypeId::Int32, - }, - Signature { - input: &[DataTypeId::Int64], - variadic: None, - return_type: DataTypeId::Int64, - }, - Signature { - input: &[DataTypeId::Int128], - variadic: None, - return_type: DataTypeId::Int128, - }, - Signature { - input: &[DataTypeId::Interval], - variadic: None, - return_type: DataTypeId::Interval, - }, - ] + const SIGS: &[Signature] = &[ + Signature::new_positional(&[DataTypeId::Float16], DataTypeId::Float16), + Signature::new_positional(&[DataTypeId::Float32], DataTypeId::Float32), + Signature::new_positional(&[DataTypeId::Float64], DataTypeId::Float64), + Signature::new_positional(&[DataTypeId::Int8], DataTypeId::Int8), + Signature::new_positional(&[DataTypeId::Int16], DataTypeId::Int16), + Signature::new_positional(&[DataTypeId::Int32], DataTypeId::Int32), + Signature::new_positional(&[DataTypeId::Int64], DataTypeId::Int64), + Signature::new_positional(&[DataTypeId::Int128], DataTypeId::Int128), + Signature::new_positional(&[DataTypeId::Interval], DataTypeId::Interval), + ]; + &SIGS } } From f96b3132df06c3b6103af27f3a4d6aacbe8be859 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Fri, 13 Dec 2024 12:03:19 -0600 Subject: [PATCH 02/23] Make table functions implement FuntionInfo --- crates/rayexec_csv/src/read_csv.rs | 14 +++- crates/rayexec_delta/src/read_delta.rs | 14 +++- .../src/functions/aggregate/builtin/avg.rs | 16 ++-- .../src/functions/aggregate/builtin/corr.rs | 4 +- .../src/functions/aggregate/builtin/count.rs | 4 +- .../src/functions/aggregate/builtin/covar.rs | 8 +- .../src/functions/aggregate/builtin/first.rs | 4 +- .../src/functions/aggregate/builtin/minmax.rs | 8 +- .../functions/aggregate/builtin/regr_avg.rs | 8 +- .../functions/aggregate/builtin/regr_count.rs | 4 +- .../functions/aggregate/builtin/regr_r2.rs | 4 +- .../functions/aggregate/builtin/regr_slope.rs | 4 +- .../src/functions/aggregate/builtin/stddev.rs | 16 ++-- .../functions/aggregate/builtin/string_agg.rs | 4 +- .../src/functions/aggregate/builtin/sum.rs | 16 ++-- crates/rayexec_execution/src/functions/mod.rs | 50 +++++++----- .../src/functions/scalar/builtin/boolean.rs | 8 +- .../functions/scalar/builtin/comparison.rs | 80 +++++++++---------- .../scalar/builtin/datetime/date_part.rs | 12 +-- .../scalar/builtin/datetime/date_trunc.rs | 12 +-- .../scalar/builtin/datetime/epoch.rs | 8 +- .../src/functions/scalar/builtin/is.rs | 24 +++--- .../scalar/builtin/list/list_extract.rs | 4 +- .../scalar/builtin/list/list_values.rs | 4 +- .../src/functions/scalar/builtin/negate.rs | 4 +- .../functions/scalar/builtin/numeric/isnan.rs | 12 +-- .../functions/scalar/builtin/numeric/mod.rs | 12 +-- .../src/functions/scalar/builtin/random.rs | 4 +- .../scalar/builtin/similarity/l2_distance.rs | 4 +- .../functions/scalar/builtin/string/ascii.rs | 4 +- .../functions/scalar/builtin/string/case.rs | 8 +- .../functions/scalar/builtin/string/concat.rs | 4 +- .../scalar/builtin/string/contains.rs | 4 +- .../scalar/builtin/string/ends_with.rs | 4 +- .../functions/scalar/builtin/string/length.rs | 20 ++--- .../functions/scalar/builtin/string/like.rs | 4 +- .../functions/scalar/builtin/string/pad.rs | 16 ++-- .../scalar/builtin/string/regexp_replace.rs | 4 +- .../functions/scalar/builtin/string/repeat.rs | 4 +- .../scalar/builtin/string/starts_with.rs | 4 +- .../scalar/builtin/string/substring.rs | 8 +- .../functions/scalar/builtin/string/trim.rs | 8 +- .../functions/scalar/builtin/struct_funcs.rs | 8 +- .../src/functions/table/builtin/refresh.rs | 15 +++- .../src/functions/table/builtin/series.rs | 22 ++++- .../src/functions/table/builtin/system.rs | 9 ++- .../src/functions/table/mod.rs | 19 ++--- crates/rayexec_iceberg/src/read_iceberg.rs | 14 +++- .../src/functions/read_parquet.rs | 14 +++- crates/rayexec_postgres/src/read_postgres.rs | 14 +++- crates/rayexec_unity_catalog/src/functions.rs | 23 +++++- 51 files changed, 361 insertions(+), 235 deletions(-) diff --git a/crates/rayexec_csv/src/read_csv.rs b/crates/rayexec_csv/src/read_csv.rs index e2c277e36..700c0400b 100644 --- a/crates/rayexec_csv/src/read_csv.rs +++ b/crates/rayexec_csv/src/read_csv.rs @@ -1,10 +1,12 @@ use futures::future::BoxFuture; use futures::StreamExt; +use rayexec_bullet::datatype::DataTypeId; use rayexec_bullet::field::Schema; use rayexec_error::{RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; use rayexec_io::location::{AccessConfig, FileLocation}; @@ -22,7 +24,7 @@ pub struct ReadCsv { pub(crate) runtime: R, } -impl TableFunction for ReadCsv { +impl FunctionInfo for ReadCsv { fn name(&self) -> &'static str { "read_csv" } @@ -31,6 +33,16 @@ impl TableFunction for ReadCsv { &["csv_scan"] } + fn signatures(&self) -> &[Signature] { + &[Signature { + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, + return_type: DataTypeId::Any, + }] + } +} + +impl TableFunction for ReadCsv { fn plan_and_initialize<'a>( &self, _context: &'a DatabaseContext, diff --git a/crates/rayexec_delta/src/read_delta.rs b/crates/rayexec_delta/src/read_delta.rs index ea255ace4..e92c5d165 100644 --- a/crates/rayexec_delta/src/read_delta.rs +++ b/crates/rayexec_delta/src/read_delta.rs @@ -1,11 +1,13 @@ use std::sync::Arc; use futures::future::BoxFuture; +use rayexec_bullet::datatype::DataTypeId; use rayexec_bullet::field::Schema; use rayexec_error::{RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; use rayexec_io::location::{AccessConfig, FileLocation}; @@ -20,7 +22,7 @@ pub struct ReadDelta { pub(crate) runtime: R, } -impl TableFunction for ReadDelta { +impl FunctionInfo for ReadDelta { fn name(&self) -> &'static str { "read_delta" } @@ -29,6 +31,16 @@ impl TableFunction for ReadDelta { &["delta_scan"] } + fn signatures(&self) -> &[Signature] { + &[Signature { + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, + return_type: DataTypeId::Any, + }] + } +} + +impl TableFunction for ReadDelta { fn plan_and_initialize<'a>( &self, _context: &'a DatabaseContext, diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/avg.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/avg.rs index 3693d8f44..84466751d 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/avg.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/avg.rs @@ -38,23 +38,23 @@ impl FunctionInfo for Avg { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }, Signature { - input: &[DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Float64, // TODO: Should be decimal // TODO: Should it though? }, Signature { - input: &[DataTypeId::Decimal64], - variadic: None, + positional_args: &[DataTypeId::Decimal64], + variadic_arg: None, return_type: DataTypeId::Float64, }, Signature { - input: &[DataTypeId::Decimal128], - variadic: None, + positional_args: &[DataTypeId::Decimal128], + variadic_arg: None, return_type: DataTypeId::Float64, }, ] diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/corr.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/corr.rs index e81038878..ddd5b2e1f 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/corr.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/corr.rs @@ -32,8 +32,8 @@ impl FunctionInfo for Corr { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/count.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/count.rs index 4acf99b5f..77e7f4185 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/count.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/count.rs @@ -39,8 +39,8 @@ impl FunctionInfo for Count { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Any], - variadic: None, + positional_args: &[DataTypeId::Any], + variadic_arg: None, return_type: DataTypeId::Int64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/covar.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/covar.rs index 3a7b15dc0..8d127b380 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/covar.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/covar.rs @@ -30,8 +30,8 @@ impl FunctionInfo for CovarPop { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } @@ -82,8 +82,8 @@ impl FunctionInfo for CovarSamp { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/first.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/first.rs index 525621bff..987326920 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/first.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/first.rs @@ -57,8 +57,8 @@ impl FunctionInfo for First { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Any], - variadic: None, + positional_args: &[DataTypeId::Any], + variadic_arg: None, return_type: DataTypeId::Any, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/minmax.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/minmax.rs index 9baa53326..ee435df14 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/minmax.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/minmax.rs @@ -57,8 +57,8 @@ impl FunctionInfo for Min { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Any], - variadic: None, + positional_args: &[DataTypeId::Any], + variadic_arg: None, return_type: DataTypeId::Any, }] } @@ -145,8 +145,8 @@ impl FunctionInfo for Max { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Any], - variadic: None, + positional_args: &[DataTypeId::Any], + variadic_arg: None, return_type: DataTypeId::Any, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_avg.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_avg.rs index 71cee24a8..c3e49c4a6 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_avg.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_avg.rs @@ -30,8 +30,8 @@ impl FunctionInfo for RegrAvgY { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } @@ -89,8 +89,8 @@ impl FunctionInfo for RegrAvgX { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_count.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_count.rs index 88ff90be5..20788f0ba 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_count.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_count.rs @@ -29,8 +29,8 @@ impl FunctionInfo for RegrCount { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Int64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_r2.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_r2.rs index a83b73391..bda32a369 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_r2.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_r2.rs @@ -30,8 +30,8 @@ impl FunctionInfo for RegrR2 { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_slope.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_slope.rs index 84c2e97d1..0172aaeca 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/regr_slope.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/regr_slope.rs @@ -31,8 +31,8 @@ impl FunctionInfo for RegrSlope { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/stddev.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/stddev.rs index 33cded517..bd88471f1 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/stddev.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/stddev.rs @@ -30,8 +30,8 @@ impl FunctionInfo for StddevPop { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } @@ -83,8 +83,8 @@ impl FunctionInfo for StddevSamp { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } @@ -132,8 +132,8 @@ impl FunctionInfo for VarPop { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } @@ -181,8 +181,8 @@ impl FunctionInfo for VarSamp { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/string_agg.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/string_agg.rs index 98e338cfc..6457eb872 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/string_agg.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/string_agg.rs @@ -29,8 +29,8 @@ impl FunctionInfo for StringAgg { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }] } diff --git a/crates/rayexec_execution/src/functions/aggregate/builtin/sum.rs b/crates/rayexec_execution/src/functions/aggregate/builtin/sum.rs index a7160629c..66f53e18b 100644 --- a/crates/rayexec_execution/src/functions/aggregate/builtin/sum.rs +++ b/crates/rayexec_execution/src/functions/aggregate/builtin/sum.rs @@ -36,23 +36,23 @@ impl FunctionInfo for Sum { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }, Signature { - input: &[DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Int64, // TODO: Should be big num }, Signature { - input: &[DataTypeId::Decimal64], - variadic: None, + positional_args: &[DataTypeId::Decimal64], + variadic_arg: None, return_type: DataTypeId::Decimal64, }, Signature { - input: &[DataTypeId::Decimal128], - variadic: None, + positional_args: &[DataTypeId::Decimal128], + variadic_arg: None, return_type: DataTypeId::Decimal128, }, ] diff --git a/crates/rayexec_execution/src/functions/mod.rs b/crates/rayexec_execution/src/functions/mod.rs index 3bd423197..83b4fe22d 100644 --- a/crates/rayexec_execution/src/functions/mod.rs +++ b/crates/rayexec_execution/src/functions/mod.rs @@ -14,10 +14,11 @@ use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_error::{RayexecError, Result}; /// Function signature. -#[derive(Debug, Clone, PartialEq)] +// TODO: Include named args. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] pub struct Signature { - /// Expected positional input types for this signature. - pub input: &'static [DataTypeId], + /// Expected positional input argument types for this signature. + pub positional_args: &'static [DataTypeId], /// Type of the variadic args if this function is variadic. /// @@ -28,37 +29,45 @@ pub struct Signature { /// match, and instead a candidate signature search will be triggered. This /// allows us to determine a single data type that all variadic args can be /// cast to, which simplifies planning and function implementation. - pub variadic: Option, + pub variadic_arg: Option, /// The expected return type. /// /// This is purely informational (and could be used for documentation). The /// concrete data type is determined by the planned function, which is what /// gets used during planning. + // TODO: Remove? pub return_type: DataTypeId, } +/// Represents a named argument in the signature. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct NamedArgument { + pub name: &'static str, + pub arg: DataTypeId, +} + impl Signature { pub const fn new_positional(input: &'static [DataTypeId], return_type: DataTypeId) -> Self { Signature { - input, - variadic: None, + positional_args: input, + variadic_arg: None, return_type, } } /// Check if this signature is a variadic signature. pub const fn is_variadic(&self) -> bool { - self.variadic.is_some() + self.variadic_arg.is_some() } /// Return if inputs given data types exactly satisfy the signature. fn exact_match(&self, inputs: &[DataType]) -> bool { - if self.input.len() != inputs.len() && !self.is_variadic() { + if self.positional_args.len() != inputs.len() && !self.is_variadic() { return false; } - for (&expected, have) in self.input.iter().zip(inputs.iter()) { + for (&expected, have) in self.positional_args.iter().zip(inputs.iter()) { if expected == DataTypeId::Any { continue; } @@ -69,8 +78,8 @@ impl Signature { } // Check variadic. - if let Some(expected) = self.variadic { - let remaining = &inputs[self.input.len()..]; + if let Some(expected) = self.variadic_arg { + let remaining = &inputs[self.positional_args.len()..]; for have in remaining { if expected == DataTypeId::Any { // If we're matching against any, we're never an exact match. @@ -168,7 +177,12 @@ impl CandidateSignature { let mut buf = Vec::new(); for (idx, sig) in sigs.iter().enumerate() { - if !Self::compare_and_fill_types(inputs, sig.input, sig.variadic, &mut buf) { + if !Self::compare_and_fill_types( + inputs, + sig.positional_args, + sig.variadic_arg, + &mut buf, + ) { continue; } @@ -358,8 +372,8 @@ mod tests { fn find_candidate_no_match() { let inputs = &[DataType::Int64]; let sigs = &[Signature { - input: &[DataTypeId::List], - variadic: None, + positional_args: &[DataTypeId::List], + variadic_arg: None, return_type: DataTypeId::Int64, }]; @@ -372,8 +386,8 @@ mod tests { fn find_candidate_simple_no_variadic() { let inputs = &[DataType::Int64]; let sigs = &[Signature { - input: &[DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Int64, }]; @@ -390,8 +404,8 @@ mod tests { fn find_candidate_simple_with_variadic() { let inputs = &[DataType::Int64, DataType::Int64, DataType::Int64]; let sigs = &[Signature { - input: &[], - variadic: Some(DataTypeId::Any), + positional_args: &[], + variadic_arg: Some(DataTypeId::Any), return_type: DataTypeId::List, }]; diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/boolean.rs b/crates/rayexec_execution/src/functions/scalar/builtin/boolean.rs index 9c49cbbb2..fd96ad324 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/boolean.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/boolean.rs @@ -25,8 +25,8 @@ impl FunctionInfo for And { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[], - variadic: Some(DataTypeId::Boolean), + positional_args: &[], + variadic_arg: Some(DataTypeId::Boolean), return_type: DataTypeId::Boolean, }] } @@ -124,8 +124,8 @@ impl FunctionInfo for Or { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[], - variadic: Some(DataTypeId::Boolean), + positional_args: &[], + variadic_arg: Some(DataTypeId::Boolean), return_type: DataTypeId::Boolean, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/comparison.rs b/crates/rayexec_execution/src/functions/scalar/builtin/comparison.rs index e61cc62e9..a60a7416e 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/comparison.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/comparison.rs @@ -44,103 +44,103 @@ use crate::logical::binder::table_list::TableList; const COMPARISON_SIGNATURES: &[Signature] = &[ Signature { - input: &[DataTypeId::Boolean, DataTypeId::Boolean], - variadic: None, + positional_args: &[DataTypeId::Boolean, DataTypeId::Boolean], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Int8, DataTypeId::Int8], - variadic: None, + positional_args: &[DataTypeId::Int8, DataTypeId::Int8], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Int16, DataTypeId::Int16], - variadic: None, + positional_args: &[DataTypeId::Int16, DataTypeId::Int16], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Int32, DataTypeId::Int32], - variadic: None, + positional_args: &[DataTypeId::Int32, DataTypeId::Int32], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Int64, DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Int64, DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Int128, DataTypeId::Int128], - variadic: None, + positional_args: &[DataTypeId::Int128, DataTypeId::Int128], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::UInt8, DataTypeId::UInt8], - variadic: None, + positional_args: &[DataTypeId::UInt8, DataTypeId::UInt8], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::UInt16, DataTypeId::UInt16], - variadic: None, + positional_args: &[DataTypeId::UInt16, DataTypeId::UInt16], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::UInt32, DataTypeId::UInt32], - variadic: None, + positional_args: &[DataTypeId::UInt32, DataTypeId::UInt32], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::UInt64, DataTypeId::UInt64], - variadic: None, + positional_args: &[DataTypeId::UInt64, DataTypeId::UInt64], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::UInt128, DataTypeId::UInt128], - variadic: None, + positional_args: &[DataTypeId::UInt128, DataTypeId::UInt128], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Float16, DataTypeId::Float16], - variadic: None, + positional_args: &[DataTypeId::Float16, DataTypeId::Float16], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Float32, DataTypeId::Float32], - variadic: None, + positional_args: &[DataTypeId::Float32, DataTypeId::Float32], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Float64, DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64, DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Decimal64, DataTypeId::Decimal64], - variadic: None, + positional_args: &[DataTypeId::Decimal64, DataTypeId::Decimal64], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Decimal128, DataTypeId::Decimal128], - variadic: None, + positional_args: &[DataTypeId::Decimal128, DataTypeId::Decimal128], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Timestamp, DataTypeId::Timestamp], - variadic: None, + positional_args: &[DataTypeId::Timestamp, DataTypeId::Timestamp], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Date32, DataTypeId::Date32], - variadic: None, + positional_args: &[DataTypeId::Date32, DataTypeId::Date32], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Binary, DataTypeId::Binary], - variadic: None, + positional_args: &[DataTypeId::Binary, DataTypeId::Binary], + variadic_arg: None, return_type: DataTypeId::Boolean, }, ]; diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_part.rs b/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_part.rs index ee340d432..400ddbdd9 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_part.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_part.rs @@ -23,18 +23,18 @@ impl FunctionInfo for DatePart { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Utf8, DataTypeId::Date32], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Date32], + variadic_arg: None, return_type: DataTypeId::Decimal64, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Date64], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Date64], + variadic_arg: None, return_type: DataTypeId::Decimal64, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Timestamp], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Timestamp], + variadic_arg: None, return_type: DataTypeId::Decimal64, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_trunc.rs b/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_trunc.rs index 48df750a5..5d356b49e 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_trunc.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/datetime/date_trunc.rs @@ -25,18 +25,18 @@ impl FunctionInfo for DateTrunc { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Utf8, DataTypeId::Date32], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Date32], + variadic_arg: None, return_type: DataTypeId::Decimal64, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Date64], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Date64], + variadic_arg: None, return_type: DataTypeId::Decimal64, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Timestamp], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Timestamp], + variadic_arg: None, return_type: DataTypeId::Decimal64, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/datetime/epoch.rs b/crates/rayexec_execution/src/functions/scalar/builtin/datetime/epoch.rs index 52d541283..87102b3a0 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/datetime/epoch.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/datetime/epoch.rs @@ -26,8 +26,8 @@ impl FunctionInfo for Epoch { &[ // S -> Timestamp Signature { - input: &[DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Timestamp, }, ] @@ -67,8 +67,8 @@ impl FunctionInfo for EpochMs { &[ // MS -> Timestamp Signature { - input: &[DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Timestamp, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/is.rs b/crates/rayexec_execution/src/functions/scalar/builtin/is.rs index 3613992b1..2be9ef75c 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/is.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/is.rs @@ -20,8 +20,8 @@ impl FunctionInfo for IsNull { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Any], - variadic: None, + positional_args: &[DataTypeId::Any], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } @@ -54,8 +54,8 @@ impl FunctionInfo for IsNotNull { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Any], - variadic: None, + positional_args: &[DataTypeId::Any], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } @@ -118,8 +118,8 @@ impl FunctionInfo for IsTrue { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Boolean], - variadic: None, + positional_args: &[DataTypeId::Boolean], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } @@ -152,8 +152,8 @@ impl FunctionInfo for IsNotTrue { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Boolean], - variadic: None, + positional_args: &[DataTypeId::Boolean], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } @@ -186,8 +186,8 @@ impl FunctionInfo for IsFalse { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Boolean], - variadic: None, + positional_args: &[DataTypeId::Boolean], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } @@ -220,8 +220,8 @@ impl FunctionInfo for IsNotFalse { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Boolean], - variadic: None, + positional_args: &[DataTypeId::Boolean], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/list/list_extract.rs b/crates/rayexec_execution/src/functions/scalar/builtin/list/list_extract.rs index b2162e754..9042ab1e7 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/list/list_extract.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/list/list_extract.rs @@ -53,8 +53,8 @@ impl FunctionInfo for ListExtract { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::List, DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::List, DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Any, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/list/list_values.rs b/crates/rayexec_execution/src/functions/scalar/builtin/list/list_values.rs index 83deeca00..8870fa16f 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/list/list_values.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/list/list_values.rs @@ -19,8 +19,8 @@ impl FunctionInfo for ListValues { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[], - variadic: Some(DataTypeId::Any), + positional_args: &[], + variadic_arg: Some(DataTypeId::Any), return_type: DataTypeId::List, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs b/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs index 0095bb341..49136c40c 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs @@ -125,8 +125,8 @@ impl FunctionInfo for Not { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Boolean], - variadic: None, + positional_args: &[DataTypeId::Boolean], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/numeric/isnan.rs b/crates/rayexec_execution/src/functions/scalar/builtin/numeric/isnan.rs index dfb16982d..8c71263dc 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/numeric/isnan.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/numeric/isnan.rs @@ -30,18 +30,18 @@ impl FunctionInfo for IsNan { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Float16], - variadic: None, + positional_args: &[DataTypeId::Float16], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Float32], - variadic: None, + positional_args: &[DataTypeId::Float32], + variadic_arg: None, return_type: DataTypeId::Boolean, }, Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Boolean, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/numeric/mod.rs b/crates/rayexec_execution/src/functions/scalar/builtin/numeric/mod.rs index fb4ad19a1..f528b19c8 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/numeric/mod.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/numeric/mod.rs @@ -57,18 +57,18 @@ use crate::logical::binder::table_list::TableList; // TODO: Include decimals. const UNARY_NUMERIC_INPUT_OUTPUT_SIGS: &[Signature] = &[ Signature { - input: &[DataTypeId::Float16], - variadic: None, + positional_args: &[DataTypeId::Float16], + variadic_arg: None, return_type: DataTypeId::Float16, }, Signature { - input: &[DataTypeId::Float32], - variadic: None, + positional_args: &[DataTypeId::Float32], + variadic_arg: None, return_type: DataTypeId::Float32, }, Signature { - input: &[DataTypeId::Float64], - variadic: None, + positional_args: &[DataTypeId::Float64], + variadic_arg: None, return_type: DataTypeId::Float64, }, ]; diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/random.rs b/crates/rayexec_execution/src/functions/scalar/builtin/random.rs index b87671630..caa41c0d5 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/random.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/random.rs @@ -24,8 +24,8 @@ impl FunctionInfo for Random { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[], - variadic: None, + positional_args: &[], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/similarity/l2_distance.rs b/crates/rayexec_execution/src/functions/scalar/builtin/similarity/l2_distance.rs index 109f48b06..cd2765f17 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/similarity/l2_distance.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/similarity/l2_distance.rs @@ -36,8 +36,8 @@ impl FunctionInfo for L2Distance { // TODO: Ideally return type would depend on the primitive type in the // list. &[Signature { - input: &[DataTypeId::List, DataTypeId::List], - variadic: None, + positional_args: &[DataTypeId::List, DataTypeId::List], + variadic_arg: None, return_type: DataTypeId::Float64, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/ascii.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/ascii.rs index 442bdda72..1dcb81327 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/ascii.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/ascii.rs @@ -20,8 +20,8 @@ impl FunctionInfo for Ascii { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Int32, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/case.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/case.rs index 23576a585..7fa3a76ac 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/case.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/case.rs @@ -20,8 +20,8 @@ impl FunctionInfo for Lower { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }] } @@ -66,8 +66,8 @@ impl FunctionInfo for Upper { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/concat.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/concat.rs index d66999414..1aa93b78d 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/concat.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/concat.rs @@ -23,8 +23,8 @@ impl FunctionInfo for Concat { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[], - variadic: Some(DataTypeId::Utf8), + positional_args: &[], + variadic_arg: Some(DataTypeId::Utf8), return_type: DataTypeId::Utf8, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/contains.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/contains.rs index 7d98cb094..dc62fcac7 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/contains.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/contains.rs @@ -22,8 +22,8 @@ impl FunctionInfo for Contains { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/ends_with.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/ends_with.rs index 46898e5ce..733b96db7 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/ends_with.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/ends_with.rs @@ -26,8 +26,8 @@ impl FunctionInfo for EndsWith { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/length.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/length.rs index f88751f29..e6cce5a7d 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/length.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/length.rs @@ -24,8 +24,8 @@ impl FunctionInfo for Length { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Int64, }] } @@ -84,13 +84,13 @@ impl FunctionInfo for ByteLength { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Int64, }, Signature { - input: &[DataTypeId::Binary], - variadic: None, + positional_args: &[DataTypeId::Binary], + variadic_arg: None, return_type: DataTypeId::Int64, }, ] @@ -146,13 +146,13 @@ impl FunctionInfo for BitLength { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Int64, }, Signature { - input: &[DataTypeId::Binary], - variadic: None, + positional_args: &[DataTypeId::Binary], + variadic_arg: None, return_type: DataTypeId::Int64, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/like.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/like.rs index 74ab9df7d..d46b63f01 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/like.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/like.rs @@ -25,8 +25,8 @@ impl FunctionInfo for Like { &[ // like(input, pattern) Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Boolean, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/pad.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/pad.rs index cfb35f683..753504531 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/pad.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/pad.rs @@ -26,13 +26,13 @@ impl FunctionInfo for LeftPad { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Utf8, DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Utf8, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Int64, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Int64, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }, ] @@ -120,13 +120,13 @@ impl FunctionInfo for RightPad { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Utf8, DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Utf8, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Int64, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Int64, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/regexp_replace.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/regexp_replace.rs index 813741b37..d15d92ef0 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/regexp_replace.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/regexp_replace.rs @@ -23,8 +23,8 @@ impl FunctionInfo for RegexpReplace { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/repeat.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/repeat.rs index 004bbb531..5b9bbb33d 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/repeat.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/repeat.rs @@ -22,8 +22,8 @@ impl FunctionInfo for Repeat { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8, DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Utf8, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/starts_with.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/starts_with.rs index 825f268e0..f171f69d8 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/starts_with.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/starts_with.rs @@ -26,8 +26,8 @@ impl FunctionInfo for StartsWith { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Boolean, }] } diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/substring.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/substring.rs index 33835495c..b1c444ef4 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/substring.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/substring.rs @@ -31,14 +31,14 @@ impl FunctionInfo for Substring { &[ // substring(, , ) Signature { - input: &[DataTypeId::Utf8, DataTypeId::Int64, DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Int64, DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Utf8, }, // substring(, ) Signature { - input: &[DataTypeId::Utf8, DataTypeId::Int64], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Int64], + variadic_arg: None, return_type: DataTypeId::Utf8, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/string/trim.rs b/crates/rayexec_execution/src/functions/scalar/builtin/string/trim.rs index 48c5aad45..df17a94c9 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/string/trim.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/string/trim.rs @@ -94,13 +94,13 @@ impl FunctionInfo for Trim { fn signatures(&self) -> &[Signature] { &[ Signature { - input: &[DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }, Signature { - input: &[DataTypeId::Utf8, DataTypeId::Utf8], - variadic: None, + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, return_type: DataTypeId::Utf8, }, ] diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/struct_funcs.rs b/crates/rayexec_execution/src/functions/scalar/builtin/struct_funcs.rs index f1484b5ff..b60fe10cd 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/struct_funcs.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/struct_funcs.rs @@ -18,8 +18,8 @@ impl FunctionInfo for StructPack { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Struct], - variadic: None, + positional_args: &[DataTypeId::Struct], + variadic_arg: None, return_type: DataTypeId::Struct, }] } @@ -45,8 +45,8 @@ impl FunctionInfo for StructExtract { fn signatures(&self) -> &[Signature] { &[Signature { - input: &[DataTypeId::Struct], - variadic: None, + positional_args: &[DataTypeId::Struct], + variadic_arg: None, return_type: DataTypeId::Any, }] } diff --git a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs b/crates/rayexec_execution/src/functions/table/builtin/refresh.rs index b2649dc75..5027d5806 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/refresh.rs @@ -3,13 +3,14 @@ use std::marker::PhantomData; use std::sync::Arc; use futures::future::BoxFuture; -use rayexec_bullet::datatype::DataType; +use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_bullet::field::{Field, Schema}; use rayexec_error::{RayexecError, Result}; use crate::database::memory_catalog::MemoryCatalog; use crate::database::DatabaseContext; use crate::functions::table::{PlannedTableFunction, TableFunction, TableFunctionInputs}; +use crate::functions::{FunctionInfo, Signature}; use crate::storage::catalog_storage::CatalogStorage; use crate::storage::table_storage::DataTable; @@ -80,11 +81,21 @@ impl Default for RefreshObjects { } } -impl TableFunction for RefreshObjects { +impl FunctionInfo for RefreshObjects { fn name(&self) -> &'static str { O::NAME } + fn signatures(&self) -> &[Signature] { + &[Signature { + positional_args: &[], + variadic_arg: None, + return_type: DataTypeId::Any, + }] + } +} + +impl TableFunction for RefreshObjects { fn plan_and_initialize<'a>( &self, _context: &'a DatabaseContext, diff --git a/crates/rayexec_execution/src/functions/table/builtin/series.rs b/crates/rayexec_execution/src/functions/table/builtin/series.rs index bca14a050..32a10cf5b 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/series.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/series.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use futures::future::BoxFuture; use rayexec_bullet::array::{Array, ArrayData}; use rayexec_bullet::batch::Batch; -use rayexec_bullet::datatype::DataType; +use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_bullet::field::{Field, Schema}; use rayexec_error::{RayexecError, Result}; use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; @@ -11,6 +11,7 @@ use serde::{Deserialize, Serialize}; use crate::database::DatabaseContext; use crate::functions::table::{PlannedTableFunction, TableFunction, TableFunctionInputs}; +use crate::functions::{FunctionInfo, Signature}; use crate::storage::table_storage::{ DataTable, DataTableScan, @@ -22,11 +23,28 @@ use crate::storage::table_storage::{ #[derive(Debug, Clone, Copy, PartialEq, Eq)] pub struct GenerateSeries; -impl TableFunction for GenerateSeries { +impl FunctionInfo for GenerateSeries { fn name(&self) -> &'static str { "generate_series" } + fn signatures(&self) -> &[Signature] { + &[ + Signature { + positional_args: &[DataTypeId::Int64, DataTypeId::Int64], + variadic_arg: None, + return_type: DataTypeId::Any, + }, + Signature { + positional_args: &[DataTypeId::Int64, DataTypeId::Int64, DataTypeId::Int64], + variadic_arg: None, + return_type: DataTypeId::Any, + }, + ] + } +} + +impl TableFunction for GenerateSeries { fn plan_and_initialize<'a>( &self, _context: &'a DatabaseContext, diff --git a/crates/rayexec_execution/src/functions/table/builtin/system.rs b/crates/rayexec_execution/src/functions/table/builtin/system.rs index 7846cc1f4..d794bd31d 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/system.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/system.rs @@ -18,6 +18,7 @@ use crate::database::catalog_entry::{CatalogEntryInner, CatalogEntryType}; use crate::database::memory_catalog::MemoryCatalog; use crate::database::{AttachInfo, DatabaseContext}; use crate::functions::table::{PlannedTableFunction, TableFunction, TableFunctionInputs}; +use crate::functions::{FunctionInfo, Signature}; use crate::storage::table_storage::{ DataTable, DataTableScan, @@ -241,11 +242,17 @@ impl SystemFunction { } } -impl TableFunction for SystemFunction { +impl FunctionInfo for SystemFunction { fn name(&self) -> &'static str { F::NAME } + fn signatures(&self) -> &[Signature] { + unimplemented!() + } +} + +impl TableFunction for SystemFunction { fn plan_and_initialize<'a>( &self, context: &'a DatabaseContext, diff --git a/crates/rayexec_execution/src/functions/table/mod.rs b/crates/rayexec_execution/src/functions/table/mod.rs index 81e0814a5..d93dffcce 100644 --- a/crates/rayexec_execution/src/functions/table/mod.rs +++ b/crates/rayexec_execution/src/functions/table/mod.rs @@ -14,6 +14,7 @@ use out::TableOutFunction; use rayexec_bullet::field::Schema; use rayexec_error::Result; +use super::FunctionInfo; use crate::database::DatabaseContext; use crate::logical::statistics::StatisticsValue; use crate::storage::table_storage::DataTable; @@ -26,15 +27,7 @@ use crate::storage::table_storage::DataTable; /// object store, etc. /// /// The specialized variant should be determined by function argument inputs. -pub trait TableFunction: Debug + Sync + Send + DynClone { - /// Name of the function. - fn name(&self) -> &'static str; - - /// Optional aliases for this function. - fn aliases(&self) -> &'static [&'static str] { - &[] - } - +pub trait TableFunction: FunctionInfo + Debug + Sync + Send + DynClone { /// Plan the table function using the provide args, and do any necessary /// initialization. /// @@ -45,7 +38,9 @@ pub trait TableFunction: Debug + Sync + Send + DynClone { &self, context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>>; + ) -> BoxFuture<'a, Result>> { + unimplemented!() + } fn initialize<'a>( &self, @@ -63,7 +58,9 @@ pub trait TableFunction: Debug + Sync + Send + DynClone { async move { Ok(state) }.boxed() } - fn decode_state(&self, state: &[u8]) -> Result>; + fn decode_state(&self, state: &[u8]) -> Result> { + unimplemented!() + } } impl Clone for Box { diff --git a/crates/rayexec_iceberg/src/read_iceberg.rs b/crates/rayexec_iceberg/src/read_iceberg.rs index 5394b493b..0b9aeec24 100644 --- a/crates/rayexec_iceberg/src/read_iceberg.rs +++ b/crates/rayexec_iceberg/src/read_iceberg.rs @@ -1,11 +1,13 @@ use std::sync::Arc; use futures::future::BoxFuture; +use rayexec_bullet::datatype::DataTypeId; use rayexec_bullet::field::Schema; use rayexec_error::{not_implemented, RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; use rayexec_io::location::{AccessConfig, FileLocation}; @@ -18,7 +20,7 @@ pub struct ReadIceberg { pub(crate) runtime: R, } -impl TableFunction for ReadIceberg { +impl FunctionInfo for ReadIceberg { fn name(&self) -> &'static str { "read_iceberg" } @@ -27,6 +29,16 @@ impl TableFunction for ReadIceberg { &["iceberg_scan"] } + fn signatures(&self) -> &[Signature] { + &[Signature { + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, + return_type: DataTypeId::Any, + }] + } +} + +impl TableFunction for ReadIceberg { fn plan_and_initialize<'a>( &self, _context: &'a DatabaseContext, diff --git a/crates/rayexec_parquet/src/functions/read_parquet.rs b/crates/rayexec_parquet/src/functions/read_parquet.rs index 0d13f4e30..920d3300e 100644 --- a/crates/rayexec_parquet/src/functions/read_parquet.rs +++ b/crates/rayexec_parquet/src/functions/read_parquet.rs @@ -2,11 +2,13 @@ use std::sync::Arc; use bytes::Bytes; use futures::future::BoxFuture; +use rayexec_bullet::datatype::DataTypeId; use rayexec_bullet::field::Schema; use rayexec_error::Result; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; @@ -24,7 +26,7 @@ pub struct ReadParquet { pub(crate) runtime: R, } -impl TableFunction for ReadParquet { +impl FunctionInfo for ReadParquet { fn name(&self) -> &'static str { "read_parquet" } @@ -33,6 +35,16 @@ impl TableFunction for ReadParquet { &["parquet_scan"] } + fn signatures(&self) -> &[Signature] { + &[Signature { + positional_args: &[DataTypeId::Utf8], + variadic_arg: None, + return_type: DataTypeId::Any, + }] + } +} + +impl TableFunction for ReadParquet { fn plan_and_initialize<'a>( &self, _context: &'a DatabaseContext, diff --git a/crates/rayexec_postgres/src/read_postgres.rs b/crates/rayexec_postgres/src/read_postgres.rs index 9039e799a..88fa8d19d 100644 --- a/crates/rayexec_postgres/src/read_postgres.rs +++ b/crates/rayexec_postgres/src/read_postgres.rs @@ -1,9 +1,11 @@ use futures::future::BoxFuture; +use rayexec_bullet::datatype::DataTypeId; use rayexec_bullet::field::Schema; use rayexec_error::{OptionExt, RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; @@ -17,11 +19,21 @@ pub struct ReadPostgres { pub(crate) runtime: R, } -impl TableFunction for ReadPostgres { +impl FunctionInfo for ReadPostgres { fn name(&self) -> &'static str { "read_postgres" } + fn signatures(&self) -> &[Signature] { + &[Signature { + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, + return_type: DataTypeId::Any, + }] + } +} + +impl TableFunction for ReadPostgres { fn plan_and_initialize<'a>( &self, _context: &'a DatabaseContext, diff --git a/crates/rayexec_unity_catalog/src/functions.rs b/crates/rayexec_unity_catalog/src/functions.rs index 0fb487dcb..c57294e67 100644 --- a/crates/rayexec_unity_catalog/src/functions.rs +++ b/crates/rayexec_unity_catalog/src/functions.rs @@ -6,12 +6,13 @@ use futures::stream::BoxStream; use futures::TryStreamExt; use rayexec_bullet::array::Array; use rayexec_bullet::batch::Batch; -use rayexec_bullet::datatype::DataType; +use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_bullet::field::{Field, Schema}; use rayexec_error::{not_implemented, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::{ DataTable, @@ -29,6 +30,8 @@ pub trait UnityObjectsOperation: { /// Name of the table function. const NAME: &'static str; + /// Function signatures. + const SIGNATURES: &[Signature]; /// State containing the catalog connection. type ConnectionState: Debug + Clone + Sync + Send; @@ -74,6 +77,11 @@ impl fmt::Debug for ListSchemasStreamState { impl UnityObjectsOperation for ListSchemasOperation { const NAME: &'static str = "unity_list_schemas"; + const SIGNATURES: &[Signature] = &[Signature { + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, + return_type: DataTypeId::Any, + }]; type ConnectionState = ListSchemasConnectionState; type StreamState = ListSchemasStreamState; @@ -147,6 +155,11 @@ impl fmt::Debug for ListTablesStreamState { impl UnityObjectsOperation for ListTablesOperation { const NAME: &'static str = "unity_list_tables"; + const SIGNATURES: &[Signature] = &[Signature { + positional_args: &[DataTypeId::Utf8, DataTypeId::Utf8, DataTypeId::Utf8], + variadic_arg: None, + return_type: DataTypeId::Any, + }]; type ConnectionState = ListTablesConnectionState; type StreamState = ListTablesStreamState; @@ -238,11 +251,17 @@ impl> UnityObjects { } } -impl> TableFunction for UnityObjects { +impl> FunctionInfo for UnityObjects { fn name(&self) -> &'static str { O::NAME } + fn signatures(&self) -> &[Signature] { + unimplemented!() + } +} + +impl> TableFunction for UnityObjects { fn plan_and_initialize<'a>( &self, context: &'a DatabaseContext, From 9b04333fdcbea56ca9955da8203bb475f60d46b7 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Fri, 13 Dec 2024 12:06:33 -0600 Subject: [PATCH 03/23] fixup! Make table functions implement FuntionInfo --- crates/rayexec_unity_catalog/src/functions.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/rayexec_unity_catalog/src/functions.rs b/crates/rayexec_unity_catalog/src/functions.rs index c57294e67..16a80c120 100644 --- a/crates/rayexec_unity_catalog/src/functions.rs +++ b/crates/rayexec_unity_catalog/src/functions.rs @@ -257,7 +257,7 @@ impl> FunctionInfo for UnityObjects &[Signature] { - unimplemented!() + O::SIGNATURES } } From 3f7572bca3159870b66a704691a6d6e02d96643f Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Fri, 13 Dec 2024 13:32:11 -0600 Subject: [PATCH 04/23] generate series in out --- crates/rayexec_csv/src/read_csv.rs | 10 +- crates/rayexec_delta/src/read_delta.rs | 10 +- .../src/execution/operators/table_function.rs | 6 +- .../rayexec_execution/src/functions/proto.rs | 4 +- .../src/functions/table/builtin/refresh.rs | 8 +- .../src/functions/table/builtin/series.rs | 197 +++++++++++++++++- .../src/functions/table/builtin/system.rs | 8 +- .../src/functions/table/inout.rs | 16 +- .../src/functions/table/mod.rs | 76 +++++-- .../src/functions/table/out.rs | 9 +- .../logical/binder/bind_query/bind_from.rs | 4 +- .../src/logical/logical_scan.rs | 4 +- .../resolver/resolved_table_function.rs | 4 +- crates/rayexec_iceberg/src/read_iceberg.rs | 10 +- .../src/functions/read_parquet.rs | 10 +- crates/rayexec_postgres/src/read_postgres.rs | 10 +- crates/rayexec_unity_catalog/src/functions.rs | 8 +- 17 files changed, 315 insertions(+), 79 deletions(-) diff --git a/crates/rayexec_csv/src/read_csv.rs b/crates/rayexec_csv/src/read_csv.rs index 700c0400b..5ceef770b 100644 --- a/crates/rayexec_csv/src/read_csv.rs +++ b/crates/rayexec_csv/src/read_csv.rs @@ -5,7 +5,7 @@ use rayexec_bullet::field::Schema; use rayexec_error::{RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; @@ -47,11 +47,11 @@ impl TableFunction for ReadCsv { &self, _context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { Box::pin(ReadCsvImpl::initialize(self.clone(), args)) } - fn decode_state(&self, state: &[u8]) -> Result> { + fn decode_state(&self, state: &[u8]) -> Result> { let state = ReadCsvState::decode(state)?; Ok(Box::new(ReadCsvImpl { func: self.clone(), @@ -112,7 +112,7 @@ impl ReadCsvImpl { async fn initialize( func: ReadCsv, args: TableFunctionInputs, - ) -> Result> { + ) -> Result> { let (location, conf) = args.try_location_and_access_config()?; let mut source = func @@ -155,7 +155,7 @@ impl ReadCsvImpl { } } -impl PlannedTableFunction for ReadCsvImpl { +impl PlannedTableFunction2 for ReadCsvImpl { fn table_function(&self) -> &dyn TableFunction { &self.func } diff --git a/crates/rayexec_delta/src/read_delta.rs b/crates/rayexec_delta/src/read_delta.rs index e92c5d165..55d15cb80 100644 --- a/crates/rayexec_delta/src/read_delta.rs +++ b/crates/rayexec_delta/src/read_delta.rs @@ -6,7 +6,7 @@ use rayexec_bullet::field::Schema; use rayexec_error::{RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; @@ -45,12 +45,12 @@ impl TableFunction for ReadDelta { &self, _context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { let func = self.clone(); Box::pin(async move { ReadDeltaImpl::initialize(func, args).await }) } - fn decode_state(&self, state: &[u8]) -> Result> { + fn decode_state(&self, state: &[u8]) -> Result> { Ok(Box::new(ReadDeltaImpl { func: self.clone(), state: ReadDeltaState::decode(state)?, @@ -99,7 +99,7 @@ impl ReadDeltaImpl { async fn initialize( func: ReadDelta, args: TableFunctionInputs, - ) -> Result> { + ) -> Result> { let (location, conf) = args.try_location_and_access_config()?; let provider = func.runtime.file_provider(); @@ -119,7 +119,7 @@ impl ReadDeltaImpl { } } -impl PlannedTableFunction for ReadDeltaImpl { +impl PlannedTableFunction2 for ReadDeltaImpl { fn reinitialize(&self) -> BoxFuture> { // TODO: Reinit table. // TODO: Needs mut diff --git a/crates/rayexec_execution/src/execution/operators/table_function.rs b/crates/rayexec_execution/src/execution/operators/table_function.rs index 89055e3cb..ddc5f1cdf 100644 --- a/crates/rayexec_execution/src/execution/operators/table_function.rs +++ b/crates/rayexec_execution/src/execution/operators/table_function.rs @@ -21,7 +21,7 @@ use super::{ }; use crate::database::DatabaseContext; use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; -use crate::functions::table::PlannedTableFunction; +use crate::functions::table::PlannedTableFunction2; use crate::proto::DatabaseProtoConv; use crate::storage::table_storage::{DataTableScan, Projections}; @@ -40,12 +40,12 @@ impl fmt::Debug for TableFunctionPartitionState { #[derive(Debug)] pub struct PhysicalTableFunction { - function: Box, + function: Box, projections: Projections, } impl PhysicalTableFunction { - pub fn new(function: Box, projections: Projections) -> Self { + pub fn new(function: Box, projections: Projections) -> Self { PhysicalTableFunction { function, projections, diff --git a/crates/rayexec_execution/src/functions/proto.rs b/crates/rayexec_execution/src/functions/proto.rs index a3e207aa6..6bd0c8a5a 100644 --- a/crates/rayexec_execution/src/functions/proto.rs +++ b/crates/rayexec_execution/src/functions/proto.rs @@ -8,7 +8,7 @@ use super::aggregate::{AggregateFunction, PlannedAggregateFunction}; use super::copy::{CopyToArgs, CopyToFunction}; use super::scalar::{PlannedScalarFunction, ScalarFunction}; use super::table::inputs::TableFunctionInputs; -use super::table::{PlannedTableFunction, TableFunction}; +use super::table::{PlannedTableFunction2, TableFunction}; use crate::database::catalog::CatalogTx; use crate::database::DatabaseContext; use crate::proto::DatabaseProtoConv; @@ -146,7 +146,7 @@ impl DatabaseProtoConv for Box { } } -impl DatabaseProtoConv for Box { +impl DatabaseProtoConv for Box { type ProtoType = rayexec_proto::generated::functions::PlannedTableFunction; fn to_proto_ctx(&self, _context: &DatabaseContext) -> Result { diff --git a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs b/crates/rayexec_execution/src/functions/table/builtin/refresh.rs index 5027d5806..7521bf3f2 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/refresh.rs @@ -9,7 +9,7 @@ use rayexec_error::{RayexecError, Result}; use crate::database::memory_catalog::MemoryCatalog; use crate::database::DatabaseContext; -use crate::functions::table::{PlannedTableFunction, TableFunction, TableFunctionInputs}; +use crate::functions::table::{PlannedTableFunction2, TableFunction, TableFunctionInputs}; use crate::functions::{FunctionInfo, Signature}; use crate::storage::catalog_storage::CatalogStorage; use crate::storage::table_storage::DataTable; @@ -100,11 +100,11 @@ impl TableFunction for RefreshObjects { &self, _context: &'a DatabaseContext, _args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { unimplemented!() } - fn decode_state(&self, _state: &[u8]) -> Result> { + fn decode_state(&self, _state: &[u8]) -> Result> { unimplemented!() } } @@ -116,7 +116,7 @@ pub struct RefreshObjectsImpl { _op: PhantomData, } -impl PlannedTableFunction for RefreshObjectsImpl { +impl PlannedTableFunction2 for RefreshObjectsImpl { fn encode_state(&self, _state: &mut Vec) -> Result<()> { Ok(()) } diff --git a/crates/rayexec_execution/src/functions/table/builtin/series.rs b/crates/rayexec_execution/src/functions/table/builtin/series.rs index 32a10cf5b..52ecf4170 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/series.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/series.rs @@ -1,16 +1,22 @@ use std::sync::Arc; +use std::task::{Context, Waker}; use futures::future::BoxFuture; use rayexec_bullet::array::{Array, ArrayData}; use rayexec_bullet::batch::Batch; use rayexec_bullet::datatype::{DataType, DataTypeId}; +use rayexec_bullet::executor::physical_type::PhysicalI64; +use rayexec_bullet::executor::scalar::UnaryExecutor; use rayexec_bullet::field::{Field, Schema}; +use rayexec_bullet::storage::PrimitiveStorage; use rayexec_error::{RayexecError, Result}; use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; use serde::{Deserialize, Serialize}; use crate::database::DatabaseContext; -use crate::functions::table::{PlannedTableFunction, TableFunction, TableFunctionInputs}; +use crate::execution::operators::{PollFinalize, PollPull, PollPush}; +use crate::functions::table::inout::{TableInOutFunction, TableInOutPartitionState}; +use crate::functions::table::{PlannedTableFunction2, TableFunction, TableFunctionInputs}; use crate::functions::{FunctionInfo, Signature}; use crate::storage::table_storage::{ DataTable, @@ -49,11 +55,11 @@ impl TableFunction for GenerateSeries { &self, _context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { Box::pin(async move { Self::plan_and_initialize_inner(args) }) } - fn decode_state(&self, state: &[u8]) -> Result> { + fn decode_state(&self, state: &[u8]) -> Result> { let mut packed = PackedDecoder::new(state); let start = packed.decode_next()?; let stop = packed.decode_next()?; @@ -65,7 +71,7 @@ impl TableFunction for GenerateSeries { impl GenerateSeries { fn plan_and_initialize_inner( args: TableFunctionInputs, - ) -> Result> { + ) -> Result> { if !args.named.is_empty() { return Err(RayexecError::new( "generate_series does not accept named arguments", @@ -107,7 +113,7 @@ pub struct GenerateSeriesI64 { step: i64, } -impl PlannedTableFunction for GenerateSeriesI64 { +impl PlannedTableFunction2 for GenerateSeriesI64 { fn encode_state(&self, state: &mut Vec) -> Result<()> { let mut packed = PackedEncoder::new(state); packed.encode_next(&self.start)?; @@ -207,3 +213,184 @@ impl DataTableScan for GenerateSeriesScan { Box::pin(async { Ok(self.generate_next()) }) } } + +#[derive(Debug, Clone)] +pub struct GenerateSeriesInOutImpl; + +impl TableInOutFunction for GenerateSeriesInOutImpl { + fn create_states( + &self, + num_partitions: usize, + ) -> Result>> { + let states: Vec<_> = (0..num_partitions) + .map(|_| { + Box::new(GenerateSeriesInOutPartitionState { + batch_size: 1024, // TODO + batch: None, + row_idx: 0, + finished: false, + params: SeriesParams::default(), + push_waker: None, + pull_waker: None, + }) as _ + }) + .collect(); + + Ok(states) + } +} + +#[derive(Debug, Clone, Default)] +struct SeriesParams { + batch_size: usize, + exhausted: bool, + + curr: i64, + stop: i64, + step: i64, +} + +impl SeriesParams { + /// Generate the next set of rows using the current parameters. + fn generate_next(&mut self) -> Array { + debug_assert!(!self.exhausted); + + let mut series: Vec = Vec::new(); + if self.curr < self.stop && self.step > 0 { + // Going up. + let mut count = 0; + while self.curr <= self.stop && count < self.batch_size { + series.push(self.curr); + self.curr += self.step; + count += 1; + } + } else if self.curr > self.stop && self.step < 0 { + // Going down. + let mut count = 0; + while self.curr >= self.stop && count < self.batch_size { + series.push(self.curr); + self.curr += self.step; + count += 1; + } + } + + if series.len() < self.batch_size { + self.exhausted = true; + } + + // Calculate the start value for the next iteration. + if let Some(last) = series.last() { + self.curr = *last + self.step; + } + + Array::new_with_array_data(DataType::Int64, PrimitiveStorage::from(series)) + } +} + +#[derive(Debug)] +pub struct GenerateSeriesInOutPartitionState { + batch_size: usize, + /// Batch we're working on. + batch: Option, + /// Row index we're on. + row_idx: usize, + /// If we're finished. + finished: bool, + /// Current params. + params: SeriesParams, + push_waker: Option, + pull_waker: Option, +} + +impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { + fn poll_push(&mut self, cx: &mut Context, batch: Batch) -> Result { + if self.batch.is_some() { + // Still processing current batch, come back later. + self.push_waker = Some(cx.waker().clone()); + if let Some(pull_waker) = self.pull_waker.take() { + pull_waker.wake(); + } + return Ok(PollPush::Pending(batch)); + } + + self.batch = Some(batch); + self.row_idx = 0; + + Ok(PollPush::Pushed) + } + + fn poll_finalize_push(&mut self, _cx: &mut Context) -> Result { + self.finished = true; + Ok(PollFinalize::Finalized) + } + + fn poll_pull(&mut self, cx: &mut Context) -> Result { + let batch = match &self.batch { + Some(batch) => batch, + None => { + if self.finished { + return Ok(PollPull::Exhausted); + } + + // No batch to work on, come back later. + self.pull_waker = Some(cx.waker().clone()); + if let Some(push_waker) = self.push_waker.take() { + push_waker.wake() + } + return Ok(PollPull::Pending); + } + }; + + if self.params.exhausted { + // Move to next row to process. + self.row_idx += 1; + + if self.row_idx >= batch.num_rows() { + // Need more input. + self.batch = None; + self.pull_waker = Some(cx.waker().clone()); + if let Some(push_waker) = self.push_waker.take() { + push_waker.wake() + } + + return Ok(PollPull::Pending); + } + + // Generate new params from row. + let start = + UnaryExecutor::value_at::(batch.column(0).unwrap(), self.row_idx)?; + let end = + UnaryExecutor::value_at::(batch.column(1).unwrap(), self.row_idx)?; + + // Use values from start/end if they're both not null. Otherwise use + // parameters that produce an empty array. + match (start, end) { + (Some(start), Some(end)) => { + self.params = SeriesParams { + batch_size: self.batch_size, + exhausted: false, + curr: start, + stop: end, + step: 1, // TODO + } + } + _ => { + self.params = SeriesParams { + batch_size: self.batch_size, + exhausted: false, + curr: 1, + stop: 0, + step: 1, + } + } + } + + // TODO: Validate params. + } + + let out = self.params.generate_next(); + let batch = Batch::try_new([out])?; + + Ok(PollPull::Computed(batch.into())) + } +} diff --git a/crates/rayexec_execution/src/functions/table/builtin/system.rs b/crates/rayexec_execution/src/functions/table/builtin/system.rs index d794bd31d..7d869d332 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/system.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/system.rs @@ -17,7 +17,7 @@ use crate::database::catalog::CatalogTx; use crate::database::catalog_entry::{CatalogEntryInner, CatalogEntryType}; use crate::database::memory_catalog::MemoryCatalog; use crate::database::{AttachInfo, DatabaseContext}; -use crate::functions::table::{PlannedTableFunction, TableFunction, TableFunctionInputs}; +use crate::functions::table::{PlannedTableFunction2, TableFunction, TableFunctionInputs}; use crate::functions::{FunctionInfo, Signature}; use crate::storage::table_storage::{ DataTable, @@ -257,7 +257,7 @@ impl TableFunction for SystemFunction { &self, context: &'a DatabaseContext, _args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { // TODO: Method on args returning an error if not empty. let databases = context @@ -280,7 +280,7 @@ impl TableFunction for SystemFunction { }) } - fn decode_state(&self, _state: &[u8]) -> Result> { + fn decode_state(&self, _state: &[u8]) -> Result> { Ok(Box::new(PlannedSystemFunction { databases: Vec::new(), function: *self, @@ -294,7 +294,7 @@ pub struct PlannedSystemFunction { function: SystemFunction, } -impl PlannedTableFunction for PlannedSystemFunction { +impl PlannedTableFunction2 for PlannedSystemFunction { fn encode_state(&self, _state: &mut Vec) -> Result<()> { Ok(()) } diff --git a/crates/rayexec_execution/src/functions/table/inout.rs b/crates/rayexec_execution/src/functions/table/inout.rs index 4cb4f9601..4ca71744a 100644 --- a/crates/rayexec_execution/src/functions/table/inout.rs +++ b/crates/rayexec_execution/src/functions/table/inout.rs @@ -1,17 +1,27 @@ use std::fmt::Debug; use std::task::Context; +use dyn_clone::DynClone; use rayexec_bullet::batch::Batch; use rayexec_error::Result; use crate::execution::operators::{PollFinalize, PollPull, PollPush}; -pub trait TableInOutFunction: Debug + Sync + Send { - fn create_states(&self, num_partitions: usize) -> Result>>; +pub trait TableInOutFunction: Debug + Sync + Send + DynClone { + fn create_states( + &self, + num_partitions: usize, + ) -> Result>>; } -pub trait TableInOutState: Debug + Sync + Send { +pub trait TableInOutPartitionState: Debug + Sync + Send { fn poll_push(&mut self, cx: &mut Context, batch: Batch) -> Result; fn poll_finalize_push(&mut self, cx: &mut Context) -> Result; fn poll_pull(&mut self, cx: &mut Context) -> Result; } + +impl Clone for Box { + fn clone(&self) -> Self { + dyn_clone::clone_box(&**self) + } +} diff --git a/crates/rayexec_execution/src/functions/table/mod.rs b/crates/rayexec_execution/src/functions/table/mod.rs index d93dffcce..ae7f533cb 100644 --- a/crates/rayexec_execution/src/functions/table/mod.rs +++ b/crates/rayexec_execution/src/functions/table/mod.rs @@ -3,6 +3,7 @@ pub mod inout; pub mod inputs; pub mod out; +use std::collections::HashMap; use std::fmt::Debug; use dyn_clone::DynClone; @@ -12,10 +13,13 @@ use inout::TableInOutFunction; use inputs::TableFunctionInputs; use out::TableOutFunction; use rayexec_bullet::field::Schema; +use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_error::Result; use super::FunctionInfo; use crate::database::DatabaseContext; +use crate::expr::Expression; +use crate::logical::binder::table_list::TableList; use crate::logical::statistics::StatisticsValue; use crate::storage::table_storage::DataTable; @@ -38,7 +42,7 @@ pub trait TableFunction: FunctionInfo + Debug + Sync + Send + DynClone { &self, context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { unimplemented!() } @@ -46,19 +50,21 @@ pub trait TableFunction: FunctionInfo + Debug + Sync + Send + DynClone { &self, _context: &'a DatabaseContext, _args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { unimplemented!() } - fn reinitialize<'a>( - &self, - _context: &'a DatabaseContext, - state: TableFunctionState, - ) -> BoxFuture<'a, Result> { - async move { Ok(state) }.boxed() + fn decode_state(&self, state: &[u8]) -> Result> { + unimplemented!() } - fn decode_state(&self, state: &[u8]) -> Result> { + fn plan<'a>( + &self, + context: &'a DatabaseContext, + table_list: &TableList, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { unimplemented!() } } @@ -83,17 +89,43 @@ impl PartialEq for dyn TableFunction + '_ { impl Eq for dyn TableFunction {} -#[derive(Debug)] -pub struct TableFunctionState { - pub table_function: Box, - pub inputs: TableFunctionInputs, - pub out_function: Option>, - pub inout_function: Option>, +#[derive(Debug, Clone)] +pub struct PlannedTableFunction { + /// The function that did the planning. + pub function: Box, + /// Unnamed positional arguments. + pub positional_inputs: Vec, + /// Named arguments. + pub named_inputs: HashMap, // Requiring constant values for named args is currently a limitation. + /// The function implementation. + pub function_impl: TableFunctionImpl, + /// Output cardinality of the function. pub cardinality: StatisticsValue, + /// Output schema of the function. pub schema: Schema, } -pub trait PlannedTableFunction: Debug + Sync + Send + DynClone { +impl PartialEq for PlannedTableFunction { + fn eq(&self, other: &Self) -> bool { + self.function == other.function + && self.positional_inputs == other.positional_inputs + && self.named_inputs == other.named_inputs + && self.schema == other.schema + } +} + +impl Eq for PlannedTableFunction {} + +#[derive(Debug, Clone)] +pub enum TableFunctionImpl { + /// Table function that produces a table as its output. + Out(Box), + /// A table function that accepts dynamic arguments and produces a table + /// output. + InOut(Box), +} + +pub trait PlannedTableFunction2: Debug + Sync + Send + DynClone { /// Reinitialize the table function, including re-opening any connections /// needed. /// @@ -127,21 +159,21 @@ pub trait PlannedTableFunction: Debug + Sync + Send + DynClone { fn datatable(&self) -> Result>; } -impl PartialEq for Box { - fn eq(&self, other: &dyn PlannedTableFunction) -> bool { +impl PartialEq for Box { + fn eq(&self, other: &dyn PlannedTableFunction2) -> bool { self.as_ref() == other } } -impl PartialEq for dyn PlannedTableFunction + '_ { - fn eq(&self, other: &dyn PlannedTableFunction) -> bool { +impl PartialEq for dyn PlannedTableFunction2 + '_ { + fn eq(&self, other: &dyn PlannedTableFunction2) -> bool { self.table_function() == other.table_function() && self.schema() == other.schema() } } -impl Eq for dyn PlannedTableFunction {} +impl Eq for dyn PlannedTableFunction2 {} -impl Clone for Box { +impl Clone for Box { fn clone(&self) -> Self { dyn_clone::clone_box(&**self) } diff --git a/crates/rayexec_execution/src/functions/table/out.rs b/crates/rayexec_execution/src/functions/table/out.rs index 3b1b56a1e..fc3c0af71 100644 --- a/crates/rayexec_execution/src/functions/table/out.rs +++ b/crates/rayexec_execution/src/functions/table/out.rs @@ -1,12 +1,13 @@ use std::fmt::Debug; +use dyn_clone::DynClone; use futures::future::BoxFuture; use rayexec_bullet::batch::Batch; use rayexec_error::Result; use crate::storage::table_storage::Projections; -pub trait TableOutFunction: Debug + Sync + Send { +pub trait TableOutFunction: Debug + Sync + Send + DynClone { fn scan( &self, projections: Projections, @@ -17,3 +18,9 @@ pub trait TableOutFunction: Debug + Sync + Send { pub trait TableOutState: Debug + Sync + Send { fn pull(&mut self) -> BoxFuture<'_, Result>>; } + +impl Clone for Box { + fn clone(&self) -> Self { + dyn_clone::clone_box(&**self) + } +} diff --git a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs index 9cbaa9dea..520a2e70d 100644 --- a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs +++ b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs @@ -9,7 +9,7 @@ use crate::database::catalog_entry::CatalogEntry; use crate::expr::column_expr::ColumnExpr; use crate::expr::comparison_expr::{ComparisonExpr, ComparisonOperator}; use crate::expr::Expression; -use crate::functions::table::PlannedTableFunction; +use crate::functions::table::PlannedTableFunction2; use crate::logical::binder::bind_context::{ BindContext, BindScopeRef, @@ -55,7 +55,7 @@ pub struct BoundBaseTable { pub struct BoundTableFunction { pub table_ref: TableRef, pub location: LocationRequirement, - pub function: Box, + pub function: Box, } #[derive(Debug, Clone, PartialEq, Eq)] diff --git a/crates/rayexec_execution/src/logical/logical_scan.rs b/crates/rayexec_execution/src/logical/logical_scan.rs index 984042519..be01af108 100644 --- a/crates/rayexec_execution/src/logical/logical_scan.rs +++ b/crates/rayexec_execution/src/logical/logical_scan.rs @@ -11,7 +11,7 @@ use super::statistics::StatisticsValue; use crate::database::catalog_entry::CatalogEntry; use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; use crate::expr::Expression; -use crate::functions::table::PlannedTableFunction; +use crate::functions::table::PlannedTableFunction2; // TODO: Probably remove view from this. // Maybe just split it all up. @@ -23,7 +23,7 @@ pub enum ScanSource { source: Arc, }, TableFunction { - function: Box, + function: Box, }, ExpressionList { rows: Vec>, diff --git a/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs b/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs index 94f8965e9..28846230b 100644 --- a/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs +++ b/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs @@ -4,7 +4,7 @@ use rayexec_proto::ProtoConv; use crate::database::DatabaseContext; use crate::functions::table::inputs::TableFunctionInputs; -use crate::functions::table::PlannedTableFunction; +use crate::functions::table::PlannedTableFunction2; use crate::proto::DatabaseProtoConv; /// A resolved table function reference. @@ -16,7 +16,7 @@ pub struct ResolvedTableFunctionReference { /// if not provided an alias. pub name: String, /// The function. - pub func: Box, + pub func: Box, // TODO: Maybe keep args here? } diff --git a/crates/rayexec_iceberg/src/read_iceberg.rs b/crates/rayexec_iceberg/src/read_iceberg.rs index 0b9aeec24..44d53e95e 100644 --- a/crates/rayexec_iceberg/src/read_iceberg.rs +++ b/crates/rayexec_iceberg/src/read_iceberg.rs @@ -6,7 +6,7 @@ use rayexec_bullet::field::Schema; use rayexec_error::{not_implemented, RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; @@ -43,12 +43,12 @@ impl TableFunction for ReadIceberg { &self, _context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { let func = self.clone(); Box::pin(async move { ReadIcebergImpl::initialize(func, args).await }) } - fn decode_state(&self, _state: &[u8]) -> Result> { + fn decode_state(&self, _state: &[u8]) -> Result> { // TODO not_implemented!("decode iceberg state") } @@ -72,7 +72,7 @@ impl ReadIcebergImpl { async fn initialize( func: ReadIceberg, args: TableFunctionInputs, - ) -> Result> { + ) -> Result> { let (location, conf) = args.try_location_and_access_config()?; let provider = func.runtime.file_provider(); @@ -92,7 +92,7 @@ impl ReadIcebergImpl { } } -impl PlannedTableFunction for ReadIcebergImpl { +impl PlannedTableFunction2 for ReadIcebergImpl { fn reinitialize(&self) -> BoxFuture> { // TODO: See delta Box::pin(async move { not_implemented!("reinit iceberg state") }) diff --git a/crates/rayexec_parquet/src/functions/read_parquet.rs b/crates/rayexec_parquet/src/functions/read_parquet.rs index 920d3300e..e0a019d09 100644 --- a/crates/rayexec_parquet/src/functions/read_parquet.rs +++ b/crates/rayexec_parquet/src/functions/read_parquet.rs @@ -7,7 +7,7 @@ use rayexec_bullet::field::Schema; use rayexec_error::Result; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; @@ -49,11 +49,11 @@ impl TableFunction for ReadParquet { &self, _context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { Box::pin(ReadParquetImpl::initialize(self.clone(), args)) } - fn decode_state(&self, state: &[u8]) -> Result> { + fn decode_state(&self, state: &[u8]) -> Result> { Ok(Box::new(ReadParquetImpl { func: self.clone(), state: ReadParquetState::decode(state)?, @@ -107,7 +107,7 @@ impl ReadParquetImpl { async fn initialize( func: ReadParquet, args: TableFunctionInputs, - ) -> Result> { + ) -> Result> { let (location, conf) = args.try_location_and_access_config()?; let mut source = func .runtime @@ -131,7 +131,7 @@ impl ReadParquetImpl { } } -impl PlannedTableFunction for ReadParquetImpl { +impl PlannedTableFunction2 for ReadParquetImpl { fn table_function(&self) -> &dyn TableFunction { &self.func } diff --git a/crates/rayexec_postgres/src/read_postgres.rs b/crates/rayexec_postgres/src/read_postgres.rs index 88fa8d19d..a3468ccd1 100644 --- a/crates/rayexec_postgres/src/read_postgres.rs +++ b/crates/rayexec_postgres/src/read_postgres.rs @@ -4,7 +4,7 @@ use rayexec_bullet::field::Schema; use rayexec_error::{OptionExt, RayexecError, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::DataTable; @@ -38,11 +38,11 @@ impl TableFunction for ReadPostgres { &self, _context: &'a DatabaseContext, args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { Box::pin(ReadPostgresImpl::initialize(self.clone(), args)) } - fn decode_state(&self, state: &[u8]) -> Result> { + fn decode_state(&self, state: &[u8]) -> Result> { Ok(Box::new(ReadPostgresImpl { func: self.clone(), state: ReadPostgresState::decode(state)?, @@ -94,7 +94,7 @@ where async fn initialize( func: ReadPostgres, args: TableFunctionInputs, - ) -> Result> { + ) -> Result> { if !args.named.is_empty() { return Err(RayexecError::new( "read_postgres does not accept named arguments", @@ -131,7 +131,7 @@ where } } -impl PlannedTableFunction for ReadPostgresImpl +impl PlannedTableFunction2 for ReadPostgresImpl where R: Runtime, { diff --git a/crates/rayexec_unity_catalog/src/functions.rs b/crates/rayexec_unity_catalog/src/functions.rs index 16a80c120..f3669825c 100644 --- a/crates/rayexec_unity_catalog/src/functions.rs +++ b/crates/rayexec_unity_catalog/src/functions.rs @@ -11,7 +11,7 @@ use rayexec_bullet::field::{Field, Schema}; use rayexec_error::{not_implemented, Result}; use rayexec_execution::database::DatabaseContext; use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction, TableFunction}; +use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::{ @@ -266,7 +266,7 @@ impl> TableFunction for UnityObjects BoxFuture<'a, Result>> { + ) -> BoxFuture<'a, Result>> { let func = self.clone(); let runtime = self.runtime.clone(); @@ -276,7 +276,7 @@ impl> TableFunction for UnityObjects Result> { + fn decode_state(&self, _state: &[u8]) -> Result> { not_implemented!("decode state for unity operation") } } @@ -287,7 +287,7 @@ pub struct UnityObjectsImpl> { state: O::ConnectionState, } -impl> PlannedTableFunction for UnityObjectsImpl { +impl> PlannedTableFunction2 for UnityObjectsImpl { fn table_function(&self) -> &dyn TableFunction { &self.func } From ba14645aa70e541bd4f973239a00f25face311c1 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Fri, 13 Dec 2024 14:38:55 -0600 Subject: [PATCH 05/23] in out operator --- .../src/execution/operators/mod.rs | 9 ++ .../src/execution/operators/table_inout.rs | 110 ++++++++++++++++++ 2 files changed, 119 insertions(+) create mode 100644 crates/rayexec_execution/src/execution/operators/table_inout.rs diff --git a/crates/rayexec_execution/src/execution/operators/mod.rs b/crates/rayexec_execution/src/execution/operators/mod.rs index 934fe9007..54bd7d486 100644 --- a/crates/rayexec_execution/src/execution/operators/mod.rs +++ b/crates/rayexec_execution/src/execution/operators/mod.rs @@ -23,6 +23,7 @@ pub mod sink; pub mod sort; pub mod source; pub mod table_function; +pub mod table_inout; pub mod ungrouped_aggregate; pub mod union; pub mod unnest; @@ -68,6 +69,7 @@ use sort::gather_sort::PhysicalGatherSort; use sort::scatter_sort::PhysicalScatterSort; use source::{SourceOperation, SourceOperator, SourcePartitionState}; use table_function::{PhysicalTableFunction, TableFunctionPartitionState}; +use table_inout::{PhysicalTableInOut, TableInOutPartitionState}; use ungrouped_aggregate::{ PhysicalUngroupedAggregate, UngroupedAggregateOperatorState, @@ -130,6 +132,7 @@ pub enum PartitionState { Simple(SimplePartitionState), Scan(ScanPartitionState), TableFunction(TableFunctionPartitionState), + TableInOut(TableInOutPartitionState), CreateSchema(CreateSchemaPartitionState), CreateView(CreateViewPartitionState), Drop(DropPartitionState), @@ -333,6 +336,7 @@ pub enum PhysicalOperator { Unnest(PhysicalUnnest), Scan(PhysicalScan), TableFunction(PhysicalTableFunction), + TableInOut(PhysicalTableInOut), Insert(PhysicalInsert), CopyTo(PhysicalCopyTo), CreateTable(PhysicalCreateTable), @@ -371,6 +375,7 @@ impl ExecutableOperator for PhysicalOperator { Self::Unnest(op) => op.create_states(context, partitions), Self::Scan(op) => op.create_states(context, partitions), Self::TableFunction(op) => op.create_states(context, partitions), + Self::TableInOut(op) => op.create_states(context, partitions), Self::Insert(op) => op.create_states(context, partitions), Self::CopyTo(op) => op.create_states(context, partitions), Self::CreateTable(op) => op.create_states(context, partitions), @@ -415,6 +420,7 @@ impl ExecutableOperator for PhysicalOperator { Self::Unnest(op) => op.poll_push(cx, partition_state, operator_state, batch), Self::Scan(op) => op.poll_push(cx, partition_state, operator_state, batch), Self::TableFunction(op) => op.poll_push(cx, partition_state, operator_state, batch), + Self::TableInOut(op) => op.poll_push(cx, partition_state, operator_state, batch), Self::Insert(op) => op.poll_push(cx, partition_state, operator_state, batch), Self::CopyTo(op) => op.poll_push(cx, partition_state, operator_state, batch), Self::CreateTable(op) => op.poll_push(cx, partition_state, operator_state, batch), @@ -460,6 +466,7 @@ impl ExecutableOperator for PhysicalOperator { Self::Unnest(op) => op.poll_finalize_push(cx, partition_state, operator_state), Self::Scan(op) => op.poll_finalize_push(cx, partition_state, operator_state), Self::TableFunction(op) => op.poll_finalize_push(cx, partition_state, operator_state), + Self::TableInOut(op) => op.poll_finalize_push(cx, partition_state, operator_state), Self::Insert(op) => op.poll_finalize_push(cx, partition_state, operator_state), Self::CopyTo(op) => op.poll_finalize_push(cx, partition_state, operator_state), Self::CreateTable(op) => op.poll_finalize_push(cx, partition_state, operator_state), @@ -499,6 +506,7 @@ impl ExecutableOperator for PhysicalOperator { Self::Unnest(op) => op.poll_pull(cx, partition_state, operator_state), Self::Scan(op) => op.poll_pull(cx, partition_state, operator_state), Self::TableFunction(op) => op.poll_pull(cx, partition_state, operator_state), + Self::TableInOut(op) => op.poll_pull(cx, partition_state, operator_state), Self::Insert(op) => op.poll_pull(cx, partition_state, operator_state), Self::CopyTo(op) => op.poll_pull(cx, partition_state, operator_state), Self::CreateTable(op) => op.poll_pull(cx, partition_state, operator_state), @@ -535,6 +543,7 @@ impl Explainable for PhysicalOperator { Self::Unnest(op) => op.explain_entry(conf), Self::Scan(op) => op.explain_entry(conf), Self::TableFunction(op) => op.explain_entry(conf), + Self::TableInOut(op) => op.explain_entry(conf), Self::Insert(op) => op.explain_entry(conf), Self::CopyTo(op) => op.explain_entry(conf), Self::CreateTable(op) => op.explain_entry(conf), diff --git a/crates/rayexec_execution/src/execution/operators/table_inout.rs b/crates/rayexec_execution/src/execution/operators/table_inout.rs new file mode 100644 index 000000000..a9293bfa4 --- /dev/null +++ b/crates/rayexec_execution/src/execution/operators/table_inout.rs @@ -0,0 +1,110 @@ +use std::sync::Arc; +use std::task::Context; + +use rayexec_bullet::batch::Batch; +use rayexec_error::{RayexecError, Result}; + +use super::{ + ExecutableOperator, + ExecutionStates, + InputOutputStates, + OperatorState, + PartitionState, + PollFinalize, + PollPull, + PollPush, +}; +use crate::database::DatabaseContext; +use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; +use crate::functions::table::{inout, PlannedTableFunction, TableFunctionImpl}; + +#[derive(Debug)] +pub struct TableInOutPartitionState { + state: Box, +} + +#[derive(Debug)] +pub struct PhysicalTableInOut { + pub function: PlannedTableFunction, +} + +impl ExecutableOperator for PhysicalTableInOut { + fn create_states( + &self, + _context: &DatabaseContext, + partitions: Vec, + ) -> Result { + let partitions = partitions[0]; + + let states = match &self.function.function_impl { + TableFunctionImpl::InOut(function) => function.create_states(partitions)?, + _ => { + return Err(RayexecError::new(format!( + "'{}' is not a table in/out function", + self.function.function.name() + ))) + } + }; + + let states: Vec<_> = states + .into_iter() + .map(|state| PartitionState::TableInOut(TableInOutPartitionState { state })) + .collect(); + + Ok(ExecutionStates { + operator_state: Arc::new(OperatorState::None), + partition_states: InputOutputStates::OneToOne { + partition_states: states, + }, + }) + } + + fn poll_push( + &self, + cx: &mut Context, + partition_state: &mut PartitionState, + _operator_state: &OperatorState, + batch: Batch, + ) -> Result { + let state = match partition_state { + PartitionState::TableInOut(state) => state, + other => panic!("invalid partition state: {other:?}"), + }; + + state.state.poll_push(cx, batch) + } + + fn poll_finalize_push( + &self, + cx: &mut Context, + partition_state: &mut PartitionState, + _operator_state: &OperatorState, + ) -> Result { + let state = match partition_state { + PartitionState::TableInOut(state) => state, + other => panic!("invalid partition state: {other:?}"), + }; + + state.state.poll_finalize_push(cx) + } + + fn poll_pull( + &self, + cx: &mut Context, + partition_state: &mut PartitionState, + _operator_state: &OperatorState, + ) -> Result { + let state = match partition_state { + PartitionState::TableInOut(state) => state, + other => panic!("invalid partition state: {other:?}"), + }; + + state.state.poll_pull(cx) + } +} + +impl Explainable for PhysicalTableInOut { + fn explain_entry(&self, _conf: ExplainConfig) -> ExplainEntry { + ExplainEntry::new("TableInOut") + } +} From 79a0a1280093e783762fc140f3aed0737b6c0838 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Fri, 13 Dec 2024 18:18:04 -0600 Subject: [PATCH 06/23] wip --- .../src/execution/operators/table_inout.rs | 139 ++++++++++++++---- .../src/execution/operators/unnest.rs | 2 +- 2 files changed, 113 insertions(+), 28 deletions(-) diff --git a/crates/rayexec_execution/src/execution/operators/table_inout.rs b/crates/rayexec_execution/src/execution/operators/table_inout.rs index a9293bfa4..9f7874e9d 100644 --- a/crates/rayexec_execution/src/execution/operators/table_inout.rs +++ b/crates/rayexec_execution/src/execution/operators/table_inout.rs @@ -1,6 +1,7 @@ use std::sync::Arc; -use std::task::Context; +use std::task::{Context, Waker}; +use rayexec_bullet::array::Array; use rayexec_bullet::batch::Batch; use rayexec_error::{RayexecError, Result}; @@ -16,16 +17,49 @@ use super::{ }; use crate::database::DatabaseContext; use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; +use crate::expr::physical::PhysicalScalarExpression; use crate::functions::table::{inout, PlannedTableFunction, TableFunctionImpl}; #[derive(Debug)] pub struct TableInOutPartitionState { - state: Box, + project_inputs: Vec, + function_inputs: Vec, + function_states: Vec>, + /// Number of rows in the input. + input_num_rows: usize, + /// Row we're currently unnesting. + current_row: usize, + /// If inputs are finished. + finished: bool, + /// Push side waker. + /// + /// Set if we still have rows to process. + push_waker: Option, + /// Pull side waker. + /// + /// Set if we've processed all rows and need more input. + pull_waker: Option, } #[derive(Debug)] pub struct PhysicalTableInOut { - pub function: PlannedTableFunction, + /// The table functions. + /// + /// Output lengths from each function may differ. If lengths differ, arrays + /// will be extended with NULL values until all arrays have the same length. + pub functions: Vec, + /// Expressions used to compute the inputs to the functions. + /// + /// Holds expressions for all functions. + pub function_expressions: Vec, + /// (offset, len) pairs for each function to jump to the right expressions + /// for a given function. + pub function_offsets: Vec<(usize, usize)>, + /// Expressions that will be projected out of the operator. + /// + /// The output of the expressions will be cross joined with the output of + /// the functions. Cross joining happens after NULL extension. + pub project_expressions: Vec, } impl ExecutableOperator for PhysicalTableInOut { @@ -36,27 +70,28 @@ impl ExecutableOperator for PhysicalTableInOut { ) -> Result { let partitions = partitions[0]; - let states = match &self.function.function_impl { - TableFunctionImpl::InOut(function) => function.create_states(partitions)?, - _ => { - return Err(RayexecError::new(format!( - "'{}' is not a table in/out function", - self.function.function.name() - ))) - } - }; + unimplemented!() + // let states = match &self.function.function_impl { + // TableFunctionImpl::InOut(function) => function.create_states(partitions)?, + // _ => { + // return Err(RayexecError::new(format!( + // "'{}' is not a table in/out function", + // self.function.function.name() + // ))) + // } + // }; + + // let states: Vec<_> = states + // .into_iter() + // .map(|state| PartitionState::TableInOut(TableInOutPartitionState { state })) + // .collect(); - let states: Vec<_> = states - .into_iter() - .map(|state| PartitionState::TableInOut(TableInOutPartitionState { state })) - .collect(); - - Ok(ExecutionStates { - operator_state: Arc::new(OperatorState::None), - partition_states: InputOutputStates::OneToOne { - partition_states: states, - }, - }) + // Ok(ExecutionStates { + // operator_state: Arc::new(OperatorState::None), + // partition_states: InputOutputStates::OneToOne { + // partition_states: states, + // }, + // }) } fn poll_push( @@ -71,7 +106,33 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid partition state: {other:?}"), }; - state.state.poll_push(cx, batch) + if state.current_row < state.input_num_rows { + // Still processing inputs, come back later. + state.push_waker = Some(cx.waker().clone()); + if let Some(waker) = state.pull_waker.take() { + waker.wake(); + } + + return Ok(PollPush::Pending(batch)); + } + + // Compute inputs. These will be stored until we've processed all rows. + for (col_idx, expr) in self.project_expressions.iter().enumerate() { + state.project_inputs[col_idx] = expr.eval(&batch)?.into_owned(); + } + + for (col_idx, expr) in self.function_expressions.iter().enumerate() { + state.function_inputs[col_idx] = expr.eval(&batch)?.into_owned(); + } + + state.input_num_rows = batch.num_rows(); + state.current_row = 0; + + if let Some(waker) = state.pull_waker.take() { + waker.wake(); + } + + Ok(PollPush::Pushed) } fn poll_finalize_push( @@ -82,10 +143,16 @@ impl ExecutableOperator for PhysicalTableInOut { ) -> Result { let state = match partition_state { PartitionState::TableInOut(state) => state, - other => panic!("invalid partition state: {other:?}"), + other => panic!("invalid state: {other:?}"), }; - state.state.poll_finalize_push(cx) + state.finished = true; + + if let Some(waker) = state.pull_waker.take() { + waker.wake(); + } + + Ok(PollFinalize::Finalized) } fn poll_pull( @@ -99,7 +166,25 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid partition state: {other:?}"), }; - state.state.poll_pull(cx) + if state.current_row >= state.input_num_rows { + if state.finished { + return Ok(PollPull::Exhausted); + } + + // We're done with these inputs. Come back later. + state.pull_waker = Some(cx.waker().clone()); + if let Some(waker) = state.push_waker.take() { + waker.wake(); + } + + return Ok(PollPull::Pending); + } + + let mut outputs = + Vec::with_capacity(state.function_inputs.len() + state.project_inputs.len()); + + // state.state.poll_pull(cx) + unimplemented!() } } diff --git a/crates/rayexec_execution/src/execution/operators/unnest.rs b/crates/rayexec_execution/src/execution/operators/unnest.rs index e254cecce..9ed303efd 100644 --- a/crates/rayexec_execution/src/execution/operators/unnest.rs +++ b/crates/rayexec_execution/src/execution/operators/unnest.rs @@ -151,7 +151,7 @@ impl ExecutableOperator for PhysicalUnnest { state.input_num_rows = batch.num_rows(); state.current_row = 0; - if let Some(waker) = state.push_waker.take() { + if let Some(waker) = state.pull_waker.take() { waker.wake(); } From 3f4b6e948feaaeee25c291533e449b39d1e13a8d Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sat, 14 Dec 2024 09:52:21 -0600 Subject: [PATCH 07/23] do simple thing for now --- .../src/execution/operators/table_inout.rs | 139 ++++-------------- 1 file changed, 31 insertions(+), 108 deletions(-) diff --git a/crates/rayexec_execution/src/execution/operators/table_inout.rs b/crates/rayexec_execution/src/execution/operators/table_inout.rs index 9f7874e9d..77e1beb15 100644 --- a/crates/rayexec_execution/src/execution/operators/table_inout.rs +++ b/crates/rayexec_execution/src/execution/operators/table_inout.rs @@ -1,7 +1,6 @@ use std::sync::Arc; -use std::task::{Context, Waker}; +use std::task::Context; -use rayexec_bullet::array::Array; use rayexec_bullet::batch::Batch; use rayexec_error::{RayexecError, Result}; @@ -22,44 +21,15 @@ use crate::functions::table::{inout, PlannedTableFunction, TableFunctionImpl}; #[derive(Debug)] pub struct TableInOutPartitionState { - project_inputs: Vec, - function_inputs: Vec, - function_states: Vec>, - /// Number of rows in the input. - input_num_rows: usize, - /// Row we're currently unnesting. - current_row: usize, - /// If inputs are finished. - finished: bool, - /// Push side waker. - /// - /// Set if we still have rows to process. - push_waker: Option, - /// Pull side waker. - /// - /// Set if we've processed all rows and need more input. - pull_waker: Option, + function_state: Box, } #[derive(Debug)] pub struct PhysicalTableInOut { - /// The table functions. - /// - /// Output lengths from each function may differ. If lengths differ, arrays - /// will be extended with NULL values until all arrays have the same length. - pub functions: Vec, + /// The table function. + pub function: PlannedTableFunction, /// Expressions used to compute the inputs to the functions. - /// - /// Holds expressions for all functions. pub function_expressions: Vec, - /// (offset, len) pairs for each function to jump to the right expressions - /// for a given function. - pub function_offsets: Vec<(usize, usize)>, - /// Expressions that will be projected out of the operator. - /// - /// The output of the expressions will be cross joined with the output of - /// the functions. Cross joining happens after NULL extension. - pub project_expressions: Vec, } impl ExecutableOperator for PhysicalTableInOut { @@ -70,28 +40,31 @@ impl ExecutableOperator for PhysicalTableInOut { ) -> Result { let partitions = partitions[0]; - unimplemented!() - // let states = match &self.function.function_impl { - // TableFunctionImpl::InOut(function) => function.create_states(partitions)?, - // _ => { - // return Err(RayexecError::new(format!( - // "'{}' is not a table in/out function", - // self.function.function.name() - // ))) - // } - // }; - - // let states: Vec<_> = states - // .into_iter() - // .map(|state| PartitionState::TableInOut(TableInOutPartitionState { state })) - // .collect(); + let states = match &self.function.function_impl { + TableFunctionImpl::InOut(function) => function.create_states(partitions)?, + _ => { + return Err(RayexecError::new(format!( + "'{}' is not a table in/out function", + self.function.function.name() + ))) + } + }; - // Ok(ExecutionStates { - // operator_state: Arc::new(OperatorState::None), - // partition_states: InputOutputStates::OneToOne { - // partition_states: states, - // }, - // }) + let states: Vec<_> = states + .into_iter() + .map(|state| { + PartitionState::TableInOut(TableInOutPartitionState { + function_state: state, + }) + }) + .collect(); + + Ok(ExecutionStates { + operator_state: Arc::new(OperatorState::None), + partition_states: InputOutputStates::OneToOne { + partition_states: states, + }, + }) } fn poll_push( @@ -106,33 +79,7 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid partition state: {other:?}"), }; - if state.current_row < state.input_num_rows { - // Still processing inputs, come back later. - state.push_waker = Some(cx.waker().clone()); - if let Some(waker) = state.pull_waker.take() { - waker.wake(); - } - - return Ok(PollPush::Pending(batch)); - } - - // Compute inputs. These will be stored until we've processed all rows. - for (col_idx, expr) in self.project_expressions.iter().enumerate() { - state.project_inputs[col_idx] = expr.eval(&batch)?.into_owned(); - } - - for (col_idx, expr) in self.function_expressions.iter().enumerate() { - state.function_inputs[col_idx] = expr.eval(&batch)?.into_owned(); - } - - state.input_num_rows = batch.num_rows(); - state.current_row = 0; - - if let Some(waker) = state.pull_waker.take() { - waker.wake(); - } - - Ok(PollPush::Pushed) + state.function_state.poll_push(cx, batch) } fn poll_finalize_push( @@ -146,13 +93,7 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid state: {other:?}"), }; - state.finished = true; - - if let Some(waker) = state.pull_waker.take() { - waker.wake(); - } - - Ok(PollFinalize::Finalized) + state.function_state.poll_finalize_push(cx) } fn poll_pull( @@ -166,25 +107,7 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid partition state: {other:?}"), }; - if state.current_row >= state.input_num_rows { - if state.finished { - return Ok(PollPull::Exhausted); - } - - // We're done with these inputs. Come back later. - state.pull_waker = Some(cx.waker().clone()); - if let Some(waker) = state.push_waker.take() { - waker.wake(); - } - - return Ok(PollPull::Pending); - } - - let mut outputs = - Vec::with_capacity(state.function_inputs.len() + state.project_inputs.len()); - - // state.state.poll_pull(cx) - unimplemented!() + state.function_state.poll_pull(cx) } } From 125c38f93384b886d0404b555790f2d9e3480a01 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sat, 14 Dec 2024 10:09:17 -0600 Subject: [PATCH 08/23] logical inout --- .../src/explain/formatter.rs | 1 + .../src/logical/logical_inout.rs | 49 +++++++++++++++++++ crates/rayexec_execution/src/logical/mod.rs | 1 + .../rayexec_execution/src/logical/operator.rs | 8 +++ 4 files changed, 59 insertions(+) create mode 100644 crates/rayexec_execution/src/logical/logical_inout.rs diff --git a/crates/rayexec_execution/src/explain/formatter.rs b/crates/rayexec_execution/src/explain/formatter.rs index 9b3ff20bd..936924d75 100644 --- a/crates/rayexec_execution/src/explain/formatter.rs +++ b/crates/rayexec_execution/src/explain/formatter.rs @@ -255,6 +255,7 @@ impl ExplainNode { LogicalOperator::MagicJoin(n) => (n.explain_entry(config), &n.children), LogicalOperator::Unnest(n) => (n.explain_entry(config), &n.children), LogicalOperator::Window(n) => (n.explain_entry(config), &n.children), + LogicalOperator::InOut(n) => (n.explain_entry(config), &n.children), LogicalOperator::MaterializationScan(n) => { // Materialization special case, walk children by get // materialization from bind context. diff --git a/crates/rayexec_execution/src/logical/logical_inout.rs b/crates/rayexec_execution/src/logical/logical_inout.rs new file mode 100644 index 000000000..18c06fd07 --- /dev/null +++ b/crates/rayexec_execution/src/logical/logical_inout.rs @@ -0,0 +1,49 @@ +use rayexec_error::Result; + +use super::binder::bind_context::BindContext; +use super::binder::table_list::TableRef; +use super::operator::{LogicalNode, Node}; +use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; +use crate::expr::Expression; +use crate::functions::table::PlannedTableFunction; + +/// A table function that accepts inputs and produces outputs. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct LogicalInOut { + /// Table ref for referencing the output of this function. + pub table_ref: TableRef, + /// The table function. + pub function: PlannedTableFunction, +} + +impl Explainable for LogicalInOut { + fn explain_entry(&self, _conf: ExplainConfig) -> ExplainEntry { + ExplainEntry::new("TableInOut").with_value("function", self.function.function.name()) + } +} + +impl LogicalNode for Node { + fn get_output_table_refs(&self, _bind_context: &BindContext) -> Vec { + vec![self.node.table_ref] + } + + fn for_each_expr(&self, func: &mut F) -> Result<()> + where + F: FnMut(&Expression) -> Result<()>, + { + for expr in &self.node.function.positional_inputs { + func(expr)? + } + Ok(()) + } + + fn for_each_expr_mut(&mut self, func: &mut F) -> Result<()> + where + F: FnMut(&mut Expression) -> Result<()>, + { + for expr in &mut self.node.function.positional_inputs { + func(expr)? + } + Ok(()) + } +} diff --git a/crates/rayexec_execution/src/logical/mod.rs b/crates/rayexec_execution/src/logical/mod.rs index 646614284..9426b77af 100644 --- a/crates/rayexec_execution/src/logical/mod.rs +++ b/crates/rayexec_execution/src/logical/mod.rs @@ -54,6 +54,7 @@ pub mod logical_drop; pub mod logical_empty; pub mod logical_explain; pub mod logical_filter; +pub mod logical_inout; pub mod logical_insert; pub mod logical_join; pub mod logical_limit; diff --git a/crates/rayexec_execution/src/logical/operator.rs b/crates/rayexec_execution/src/logical/operator.rs index ba227c0a7..0bd583371 100644 --- a/crates/rayexec_execution/src/logical/operator.rs +++ b/crates/rayexec_execution/src/logical/operator.rs @@ -15,6 +15,7 @@ use super::logical_drop::LogicalDrop; use super::logical_empty::LogicalEmpty; use super::logical_explain::LogicalExplain; use super::logical_filter::LogicalFilter; +use super::logical_inout::LogicalInOut; use super::logical_insert::LogicalInsert; use super::logical_join::{ LogicalArbitraryJoin, @@ -291,6 +292,7 @@ pub enum LogicalOperator { MagicJoin(Node), Unnest(Node), Window(Node), + InOut(Node), } impl LogicalOperator { @@ -388,6 +390,7 @@ impl LogicalOperator { Self::MagicJoin(n) => &n.children, Self::Unnest(n) => &n.children, Self::Window(n) => &n.children, + Self::InOut(n) => &n.children, } } @@ -424,6 +427,7 @@ impl LogicalOperator { Self::MagicJoin(n) => &mut n.children, Self::Unnest(n) => &mut n.children, Self::Window(n) => &mut n.children, + Self::InOut(n) => &mut n.children, } } @@ -464,6 +468,7 @@ impl LogicalOperator { LogicalOperator::MagicJoin(n) => n.estimated_cardinality, LogicalOperator::Unnest(n) => n.estimated_cardinality, LogicalOperator::Window(n) => n.estimated_cardinality, + LogicalOperator::InOut(n) => n.estimated_cardinality, } } } @@ -502,6 +507,7 @@ impl LogicalNode for LogicalOperator { LogicalOperator::MagicJoin(n) => n.get_output_table_refs(bind_context), LogicalOperator::Unnest(n) => n.get_output_table_refs(bind_context), LogicalOperator::Window(n) => n.get_output_table_refs(bind_context), + LogicalOperator::InOut(n) => n.get_output_table_refs(bind_context), } } @@ -541,6 +547,7 @@ impl LogicalNode for LogicalOperator { LogicalOperator::MagicJoin(n) => n.for_each_expr(func), LogicalOperator::Unnest(n) => n.for_each_expr(func), LogicalOperator::Window(n) => n.for_each_expr(func), + LogicalOperator::InOut(n) => n.for_each_expr(func), } } @@ -580,6 +587,7 @@ impl LogicalNode for LogicalOperator { LogicalOperator::MagicJoin(n) => n.for_each_expr_mut(func), LogicalOperator::Unnest(n) => n.for_each_expr_mut(func), LogicalOperator::Window(n) => n.for_each_expr_mut(func), + LogicalOperator::InOut(n) => n.for_each_expr_mut(func), } } } From a9071c2d877b00c3fbdbd042547b05a6915748a3 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sat, 14 Dec 2024 10:42:22 -0600 Subject: [PATCH 09/23] plan physical --- .../src/execution/intermediate/planner/mod.rs | 2 + .../intermediate/planner/plan_inout.rs | 40 +++++++++++++++++++ .../src/execution/operators/table_inout.rs | 4 +- 3 files changed, 44 insertions(+), 2 deletions(-) create mode 100644 crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs diff --git a/crates/rayexec_execution/src/execution/intermediate/planner/mod.rs b/crates/rayexec_execution/src/execution/intermediate/planner/mod.rs index bfb9b2d27..e6cfe11c1 100644 --- a/crates/rayexec_execution/src/execution/intermediate/planner/mod.rs +++ b/crates/rayexec_execution/src/execution/intermediate/planner/mod.rs @@ -9,6 +9,7 @@ mod plan_drop; mod plan_empty; mod plan_explain; mod plan_filter; +mod plan_inout; mod plan_insert; mod plan_join; mod plan_limit; @@ -264,6 +265,7 @@ impl<'a> IntermediatePipelineBuildState<'a> { LogicalOperator::SetOp(setop) => { self.plan_set_operation(id_gen, materializations, setop) } + LogicalOperator::InOut(inout) => self.plan_inout(id_gen, materializations, inout), LogicalOperator::SetVar(_) => { Err(RayexecError::new("SET should be handled in the session")) } diff --git a/crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs b/crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs new file mode 100644 index 000000000..7b9bfe3e5 --- /dev/null +++ b/crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs @@ -0,0 +1,40 @@ +use std::sync::Arc; + +use rayexec_error::{Result, ResultExt}; + +use super::{IntermediatePipelineBuildState, Materializations, PipelineIdGen}; +use crate::execution::intermediate::pipeline::IntermediateOperator; +use crate::execution::operators::table_inout::PhysicalTableInOut; +use crate::execution::operators::PhysicalOperator; +use crate::logical::logical_inout::LogicalInOut; +use crate::logical::operator::{LogicalNode, Node}; + +impl IntermediatePipelineBuildState<'_> { + pub fn plan_inout( + &mut self, + id_gen: &mut PipelineIdGen, + materializations: &mut Materializations, + mut inout: Node, + ) -> Result<()> { + let input = inout.take_one_child_exact()?; + let input_refs = input.get_output_table_refs(self.bind_context); + self.walk(materializations, id_gen, input)?; + + let function_inputs = self + .expr_planner + .plan_scalars(&input_refs, &inout.node.function.positional_inputs) + .context("Failed to plan expressions for table inout")?; + + let operator = IntermediateOperator { + operator: Arc::new(PhysicalOperator::TableInOut(PhysicalTableInOut { + function: inout.node.function, + function_inputs, + })), + partitioning_requirement: None, + }; + + self.push_intermediate_operator(operator, inout.location, id_gen)?; + + Ok(()) + } +} diff --git a/crates/rayexec_execution/src/execution/operators/table_inout.rs b/crates/rayexec_execution/src/execution/operators/table_inout.rs index 77e1beb15..d70757334 100644 --- a/crates/rayexec_execution/src/execution/operators/table_inout.rs +++ b/crates/rayexec_execution/src/execution/operators/table_inout.rs @@ -28,8 +28,8 @@ pub struct TableInOutPartitionState { pub struct PhysicalTableInOut { /// The table function. pub function: PlannedTableFunction, - /// Expressions used to compute the inputs to the functions. - pub function_expressions: Vec, + /// Input expressions to the table function. + pub function_inputs: Vec, } impl ExecutableOperator for PhysicalTableInOut { From a8bf3bb5915bb96089479d7cefb9f63db7a5955c Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sat, 14 Dec 2024 14:55:24 -0600 Subject: [PATCH 10/23] some wip --- .../src/functions/table/mod.rs | 47 ++++-- .../src/functions/table/{out.rs => scan.rs} | 8 +- .../logical/binder/bind_query/bind_from.rs | 72 +++++---- .../src/logical/binder/constant_binder.rs | 85 +++++++++++ .../src/logical/binder/mod.rs | 1 + .../src/logical/planner/plan_unnest.rs | 5 + .../src/logical/resolver/expr_resolver.rs | 16 +- .../src/logical/resolver/mod.rs | 137 ++++++++++++------ .../src/logical/resolver/resolve_hybrid.rs | 31 ++-- .../resolver/resolved_table_function.rs | 91 +++++++----- crates/rayexec_parser/src/ast/from.rs | 2 +- crates/rayexec_parser/src/meta.rs | 4 - 12 files changed, 348 insertions(+), 151 deletions(-) rename crates/rayexec_execution/src/functions/table/{out.rs => scan.rs} (68%) create mode 100644 crates/rayexec_execution/src/logical/binder/constant_binder.rs diff --git a/crates/rayexec_execution/src/functions/table/mod.rs b/crates/rayexec_execution/src/functions/table/mod.rs index ae7f533cb..e7cd55321 100644 --- a/crates/rayexec_execution/src/functions/table/mod.rs +++ b/crates/rayexec_execution/src/functions/table/mod.rs @@ -1,7 +1,7 @@ pub mod builtin; pub mod inout; pub mod inputs; -pub mod out; +pub mod scan; use std::collections::HashMap; use std::fmt::Debug; @@ -11,10 +11,10 @@ use futures::future::BoxFuture; use futures::FutureExt; use inout::TableInOutFunction; use inputs::TableFunctionInputs; -use out::TableOutFunction; use rayexec_bullet::field::Schema; use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_error::Result; +use scan::TableScanFunction; use super::FunctionInfo; use crate::database::DatabaseContext; @@ -58,13 +58,7 @@ pub trait TableFunction: FunctionInfo + Debug + Sync + Send + DynClone { unimplemented!() } - fn plan<'a>( - &self, - context: &'a DatabaseContext, - table_list: &TableList, - positional_inputs: Vec, - named_inputs: HashMap, - ) -> Result { + fn planner(&self) -> TableFunctionPlanner { unimplemented!() } } @@ -89,6 +83,36 @@ impl PartialEq for dyn TableFunction + '_ { impl Eq for dyn TableFunction {} +#[derive(Debug)] +pub enum TableFunctionPlanner<'a> { + InOut(&'a dyn InOutPlanner), + Scan(&'a dyn ScanPlanner), +} + +pub trait InOutPlanner: Debug { + /// Plans an in/out function with possibly dynamic positional inputs. + fn plan( + &self, + table_list: &TableList, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result; +} + +pub trait ScanPlanner: Debug { + /// Plans an table scan function. + /// + /// This only accepts constant arguments as it's meant to be used when + /// reading tables from an external resource. Functions like `read_parquet` + /// or `read_postgres` should implement this. + fn plan<'a>( + &self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result>; +} + #[derive(Debug, Clone)] pub struct PlannedTableFunction { /// The function that did the planning. @@ -98,6 +122,9 @@ pub struct PlannedTableFunction { /// Named arguments. pub named_inputs: HashMap, // Requiring constant values for named args is currently a limitation. /// The function implementation. + /// + /// The variant used here should match the variant of the planner that + /// `function` returns from its `planner` method. pub function_impl: TableFunctionImpl, /// Output cardinality of the function. pub cardinality: StatisticsValue, @@ -119,7 +146,7 @@ impl Eq for PlannedTableFunction {} #[derive(Debug, Clone)] pub enum TableFunctionImpl { /// Table function that produces a table as its output. - Out(Box), + Scan(Box), /// A table function that accepts dynamic arguments and produces a table /// output. InOut(Box), diff --git a/crates/rayexec_execution/src/functions/table/out.rs b/crates/rayexec_execution/src/functions/table/scan.rs similarity index 68% rename from crates/rayexec_execution/src/functions/table/out.rs rename to crates/rayexec_execution/src/functions/table/scan.rs index fc3c0af71..b4bcea735 100644 --- a/crates/rayexec_execution/src/functions/table/out.rs +++ b/crates/rayexec_execution/src/functions/table/scan.rs @@ -7,19 +7,19 @@ use rayexec_error::Result; use crate::storage::table_storage::Projections; -pub trait TableOutFunction: Debug + Sync + Send + DynClone { +pub trait TableScanFunction: Debug + Sync + Send + DynClone { fn scan( &self, projections: Projections, num_partitions: usize, - ) -> Result>>; + ) -> Result>>; } -pub trait TableOutState: Debug + Sync + Send { +pub trait TableScanState: Debug + Sync + Send { fn pull(&mut self) -> BoxFuture<'_, Result>>; } -impl Clone for Box { +impl Clone for Box { fn clone(&self) -> Self { dyn_clone::clone_box(&**self) } diff --git a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs index 520a2e70d..ea9bb7b14 100644 --- a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs +++ b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs @@ -9,7 +9,7 @@ use crate::database::catalog_entry::CatalogEntry; use crate::expr::column_expr::ColumnExpr; use crate::expr::comparison_expr::{ComparisonExpr, ComparisonOperator}; use crate::expr::Expression; -use crate::functions::table::PlannedTableFunction2; +use crate::functions::table::{PlannedTableFunction, PlannedTableFunction2}; use crate::logical::binder::bind_context::{ BindContext, BindScopeRef, @@ -24,6 +24,7 @@ use crate::logical::logical_join::JoinType; use crate::logical::operator::LocationRequirement; use crate::logical::resolver::resolve_context::ResolveContext; use crate::logical::resolver::resolved_table::ResolvedTableOrCteReference; +use crate::logical::resolver::resolved_table_function::ResolvedTableFunctionReference; use crate::logical::resolver::{ResolvedMeta, ResolvedSubqueryOptions}; #[derive(Debug, Clone, PartialEq, Eq)] @@ -55,7 +56,7 @@ pub struct BoundBaseTable { pub struct BoundTableFunction { pub table_ref: TableRef, pub location: LocationRequirement, - pub function: Box, + pub function: PlannedTableFunction, } #[derive(Debug, Clone, PartialEq, Eq)] @@ -390,38 +391,47 @@ impl<'a> FromBinder<'a> { .table_functions .try_get_bound(function.reference)?; - // TODO: For table funcs that are reading files, it'd be nice to have - // the default alias be the base file path, not the function name. - let default_alias = TableAlias { - database: None, - schema: None, - table: reference.name.clone(), - }; + match reference { + ResolvedTableFunctionReference::InOut(_) => { + // Handle in/out function planning now. We have everything we + // need to plan its inputs. + let expr_binder = BaseExpressionBinder::new(self.current, self.resolve_context); + unimplemented!() + } + ResolvedTableFunctionReference::Scan(planned) => { + // TODO: For table funcs that are reading files, it'd be nice to have + // the default alias be the base file path, not the function name. + let default_alias = TableAlias { + database: None, + schema: None, + table: reference.base_table_alias(), + }; - let (names, types) = reference - .func - .schema() - .fields - .iter() - .map(|f| (f.name.clone(), f.datatype.clone())) - .unzip(); + let (names, types) = planned + .schema + .fields + .iter() + .map(|f| (f.name.clone(), f.datatype.clone())) + .unzip(); - let table_ref = self.push_table_scope_with_from_alias( - bind_context, - Some(default_alias), - names, - types, - alias, - )?; + let table_ref = self.push_table_scope_with_from_alias( + bind_context, + Some(default_alias), + names, + types, + alias, + )?; - Ok(BoundFrom { - bind_ref: self.current, - item: BoundFromItem::TableFunction(BoundTableFunction { - table_ref, - location, - function: reference.func.clone(), - }), - }) + Ok(BoundFrom { + bind_ref: self.current, + item: BoundFromItem::TableFunction(BoundTableFunction { + table_ref, + location, + function: planned.clone(), + }), + }) + } + } } fn bind_join( diff --git a/crates/rayexec_execution/src/logical/binder/constant_binder.rs b/crates/rayexec_execution/src/logical/binder/constant_binder.rs new file mode 100644 index 000000000..d476bfd92 --- /dev/null +++ b/crates/rayexec_execution/src/logical/binder/constant_binder.rs @@ -0,0 +1,85 @@ +use std::collections::HashMap; + +use rayexec_bullet::scalar::OwnedScalarValue; +use rayexec_error::{RayexecError, Result}; +use rayexec_parser::ast; + +use super::expr_binder::RecursionContext; +use crate::logical::binder::bind_context::BindContext; +use crate::logical::binder::column_binder::ErroringColumnBinder; +use crate::logical::binder::expr_binder::BaseExpressionBinder; +use crate::logical::resolver::resolve_context::ResolveContext; +use crate::logical::resolver::resolved_table_function::ConstantFunctionArgs; +use crate::logical::resolver::ResolvedMeta; +use crate::optimizer::expr_rewrite::const_fold::ConstFold; +use crate::optimizer::expr_rewrite::ExpressionRewriteRule; + +#[derive(Debug)] +pub struct ConstantBinder<'a> { + pub resolve_context: &'a ResolveContext, +} + +impl<'a> ConstantBinder<'a> { + pub fn new(resolve_context: &'a ResolveContext) -> Self { + ConstantBinder { resolve_context } + } + + /// Try to bind an AST expression as a constant value. + pub fn bind_constant_expression( + &self, + expr: &ast::Expr, + ) -> Result { + // TODO: Probably want to check that we didn't bind a subquery. + let mut bind_context = BindContext::new(); + let expr = BaseExpressionBinder::new(bind_context.root_scope_ref(), self.resolve_context) + .bind_expression( + &mut bind_context, + expr, + &mut ErroringColumnBinder, + RecursionContext { + allow_aggregates: false, + allow_windows: false, + is_root: true, + }, + )?; + + let val = ConstFold::rewrite(bind_context.get_table_list(), expr)?.try_into_scalar()?; + + Ok(val) + } + + pub fn bind_constant_function_args( + &self, + args: &[ast::FunctionArg], + ) -> Result { + let mut positional = Vec::new(); + let mut named = HashMap::new(); + + for arg in args { + match arg { + ast::FunctionArg::Named { name, arg } => { + let expr = self.bind_constant_function_arg_expr(arg)?; + named.insert(name.as_normalized_string(), expr); + } + ast::FunctionArg::Unnamed { arg } => { + let expr = self.bind_constant_function_arg_expr(arg)?; + positional.push(expr); + } + } + } + + Ok(ConstantFunctionArgs { positional, named }) + } + + fn bind_constant_function_arg_expr( + &self, + arg: &ast::FunctionArgExpr, + ) -> Result { + match arg { + ast::FunctionArgExpr::Expr(expr) => self.bind_constant_expression(expr), + ast::FunctionArgExpr::Wildcard => Err(RayexecError::new( + "'*' cannot be used as a constant function argument", + )), + } + } +} diff --git a/crates/rayexec_execution/src/logical/binder/mod.rs b/crates/rayexec_execution/src/logical/binder/mod.rs index 7cbf2ec50..6d3ff3620 100644 --- a/crates/rayexec_execution/src/logical/binder/mod.rs +++ b/crates/rayexec_execution/src/logical/binder/mod.rs @@ -12,5 +12,6 @@ pub mod bind_query; pub mod bind_set; pub mod bind_statement; pub mod column_binder; +pub mod constant_binder; pub mod expr_binder; pub mod table_list; diff --git a/crates/rayexec_execution/src/logical/planner/plan_unnest.rs b/crates/rayexec_execution/src/logical/planner/plan_unnest.rs index b3d34ac71..bb6ab25b4 100644 --- a/crates/rayexec_execution/src/logical/planner/plan_unnest.rs +++ b/crates/rayexec_execution/src/logical/planner/plan_unnest.rs @@ -9,6 +9,11 @@ use crate::logical::logical_unnest::LogicalUnnest; use crate::logical::operator::{LocationRequirement, LogicalNode, LogicalOperator, Node}; use crate::logical::statistics::StatisticsValue; +// TODO: This should be extended to support arbitrary table functions. +// +// - Left lateral join between function inputs, and the function itself. +// - Physical plan would need to handle multiple functions at once (similar to +// the current unnest plan). #[derive(Debug)] pub struct UnnestPlanner; diff --git a/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs b/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs index f6908423d..b04177983 100644 --- a/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs +++ b/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs @@ -6,6 +6,7 @@ use rayexec_parser::meta::Raw; use super::resolve_normal::create_user_facing_resolve_err; use super::resolved_function::{ResolvedFunction, SpecialBuiltinFunction}; +use super::resolved_table_function::ConstantFunctionArgs; use super::{ResolveContext, ResolvedMeta, Resolver}; use crate::database::catalog_entry::CatalogEntryType; use crate::functions::table::inputs::TableFunctionInputs; @@ -88,18 +89,15 @@ impl<'a> ExpressionResolver<'a> { }) } - /// Resolves functions arguments for a table function. + /// Resolves constant function arguments for a table function. /// /// Slightly different from normal argument resolving since arguments to a /// table function are more restrictive. E.g. we only allow literals as - /// arguments. - /// - /// Note in the future we could allow more complex expressions as arguments, - /// and we could support table function that accept columns as inputs. - pub async fn resolve_table_function_args( + /// arguments for scan table functions. + pub async fn resolve_constant_table_function_args( &self, args: Vec>, - ) -> Result { + ) -> Result { let resolve_context = &mut ResolveContext::default(); // Empty resolve context since we don't allow complex expressions. let mut named = HashMap::new(); @@ -159,7 +157,7 @@ impl<'a> ExpressionResolver<'a> { } } - Ok(TableFunctionInputs { named, positional }) + Ok(ConstantFunctionArgs { named, positional }) } pub async fn resolve_expressions( @@ -658,7 +656,7 @@ impl<'a> ExpressionResolver<'a> { )) } - async fn resolve_function_args( + pub(crate) async fn resolve_function_args( &self, args: Vec>, resolve_context: &mut ResolveContext, diff --git a/crates/rayexec_execution/src/logical/resolver/mod.rs b/crates/rayexec_execution/src/logical/resolver/mod.rs index 82d0bffe5..57bdd819f 100644 --- a/crates/rayexec_execution/src/logical/resolver/mod.rs +++ b/crates/rayexec_execution/src/logical/resolver/mod.rs @@ -16,7 +16,7 @@ use rayexec_bullet::scalar::decimal::{Decimal128Type, Decimal64Type, DecimalType use rayexec_bullet::scalar::{OwnedScalarValue, ScalarValue}; use rayexec_error::{OptionExt, RayexecError, Result}; use rayexec_io::location::FileLocation; -use rayexec_parser::ast::{self, ColumnDef, ObjectReference}; +use rayexec_parser::ast::{self, ColumnDef, FunctionArg, ObjectReference}; use rayexec_parser::meta::{AstMeta, Raw}; use rayexec_parser::parser; use rayexec_parser::statement::{RawStatement, Statement}; @@ -28,6 +28,7 @@ use resolved_table::ResolvedTableOrCteReference; use resolved_table_function::{ResolvedTableFunctionReference, UnresolvedTableFunctionReference}; use serde::{Deserialize, Serialize}; +use super::binder::constant_binder::ConstantBinder; use super::binder::expr_binder::BaseExpressionBinder; use super::binder::table_list::TableAlias; use crate::database::builtin_views::{ @@ -43,6 +44,7 @@ use crate::datasource::FileHandlers; use crate::functions::copy::CopyToArgs; use crate::functions::proto::FUNCTION_LOOKUP_CATALOG; use crate::functions::table::inputs::TableFunctionInputs; +use crate::functions::table::TableFunctionPlanner; use crate::logical::operator::LocationRequirement; /// An AST statement with references bound to data inside of the `resolve_context`. @@ -59,10 +61,6 @@ impl AstMeta for ResolvedMeta { type TableReference = ResolveListIdx; /// Index into the table functions bind list in bind data type TableFunctionReference = ResolveListIdx; - // TODO: Having this be the actual table function args does require that we - // clone them, and the args that go back into the ast don't actually do - // anything, they're never referenced again. - type TableFunctionArgs = TableFunctionInputs; /// Index into the functions bind list in bind data. type FunctionReference = ResolveListIdx; type SubqueryOptions = ResolvedSubqueryOptions; @@ -941,26 +939,40 @@ impl<'a> Resolver<'a> { ast::FromNodeBody::File(ast::FromFilePath { path }) => { match self.file_handlers.find_match(&path) { Some(handler) => { - let args = TableFunctionInputs { - named: HashMap::new(), - positional: vec![OwnedScalarValue::Utf8(path.into())], + // "Rewrite" this into a function call. + + // TODO: User-friendly alias here. + + // This isn't really needed for the typical scan case, + // but I have no idea for in/out. + let args = vec![ast::FunctionArg::Unnamed { + arg: ast::FunctionArgExpr::Expr(ast::Expr::Literal( + ast::Literal::SingleQuotedString(path.clone()), + )), + }]; + + // Having an in/out function here would be weird, but + // might as well handle it. + let resolved = match handler.table_func.planner() { + TableFunctionPlanner::InOut(_) => { + ResolvedTableFunctionReference::InOut(handler.table_func.clone()) + } + TableFunctionPlanner::Scan(planner) => { + let planned = planner + .plan(self.context, vec![path.into()], HashMap::new()) + .await?; + + ResolvedTableFunctionReference::Scan(planned) + } }; - let name = handler.table_func.name().to_string(); - let func = handler - .table_func - .plan_and_initialize(self.context, args.clone()) - .await?; - - let resolve_idx = resolve_context.table_functions.push_resolved( - ResolvedTableFunctionReference { name, func }, - LocationRequirement::ClientLocal, - ); + let resolve_idx = resolve_context + .table_functions + .push_resolved(resolved, LocationRequirement::ClientLocal); ast::FromNodeBody::TableFunction(ast::FromTableFunction { lateral: false, reference: resolve_idx, - // TODO: Not needed. args, }) } @@ -976,46 +988,77 @@ impl<'a> Resolver<'a> { reference, args, }) => { - let args = ExpressionResolver::new(self) - .resolve_table_function_args(args) - .await?; + let args = Box::pin( + ExpressionResolver::new(self).resolve_function_args(args, resolve_context), + ) + .await?; let function = match self.resolve_mode { ResolveMode::Normal => { let function = NormalResolver::new(self.tx, self.context) .require_resolve_table_function(&reference)?; - let function = function - .plan_and_initialize(self.context, args.clone()) - .await?; - MaybeResolved::Resolved( - ResolvedTableFunctionReference { - name: function.table_function().name().to_string(), - func: function, - }, - LocationRequirement::ClientLocal, - ) + let resolved = match function.planner() { + TableFunctionPlanner::InOut(_) => { + ResolvedTableFunctionReference::InOut(function) + } + TableFunctionPlanner::Scan(planner) => { + // Requires constants. + let binder = ConstantBinder::new(resolve_context); + let constant_args = binder.bind_constant_function_args(&args)?; + + let planned = planner + .plan( + self.context, + constant_args.positional, + constant_args.named, + ) + .await?; + + ResolvedTableFunctionReference::Scan(planned) + } + }; + + MaybeResolved::Resolved(resolved, LocationRequirement::ClientLocal) } ResolveMode::Hybrid => { match NormalResolver::new(self.tx, self.context) .resolve_table_function(&reference)? { Some(function) => { - let function = function - .plan_and_initialize(self.context, args.clone()) - .await?; - MaybeResolved::Resolved( - ResolvedTableFunctionReference { - name: function.table_function().name().to_string(), - func: function, - }, - LocationRequirement::ClientLocal, - ) + // TODO: Duplicated + let resolved = match function.planner() { + TableFunctionPlanner::InOut(_) => { + ResolvedTableFunctionReference::InOut(function) + } + TableFunctionPlanner::Scan(planner) => { + let binder = ConstantBinder::new(resolve_context); + let constant_args = + binder.bind_constant_function_args(&args)?; + + let planned = planner + .plan( + self.context, + constant_args.positional, + constant_args.named, + ) + .await?; + + ResolvedTableFunctionReference::Scan(planned) + } + }; + + MaybeResolved::Resolved(resolved, LocationRequirement::ClientLocal) + } + None => { + let binder = ConstantBinder::new(resolve_context); + let constant_args = binder.bind_constant_function_args(&args)?; + + MaybeResolved::Unresolved(UnresolvedTableFunctionReference { + reference, + args: constant_args, + }) } - None => MaybeResolved::Unresolved(UnresolvedTableFunctionReference { - reference, - args: args.clone(), - }), } } }; @@ -1026,8 +1069,6 @@ impl<'a> Resolver<'a> { ast::FromNodeBody::TableFunction(ast::FromTableFunction { lateral, reference: resolve_idx, - // TODO: These args aren't actually needed when bound. Not - // sure completely sure what we want to do here. args, }) } diff --git a/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs b/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs index 84e5824f7..1afed9ae7 100644 --- a/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs +++ b/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs @@ -1,3 +1,4 @@ +use std::collections::HashMap; use std::sync::Arc; use rayexec_error::{RayexecError, Result}; @@ -11,6 +12,8 @@ use crate::database::catalog::CatalogTx; use crate::database::memory_catalog::MemoryCatalog; use crate::database::{Database, DatabaseContext}; use crate::datasource::{DataSourceRegistry, FileHandlers}; +use crate::functions::table::TableFunctionPlanner; +use crate::logical::binder::constant_binder::ConstantBinder; use crate::logical::operator::LocationRequirement; use crate::logical::resolver::ResolveMode; @@ -186,19 +189,27 @@ impl<'a> HybridResolver<'a> { ) -> Result<()> { for item in resolve_context.table_functions.inner.iter_mut() { if let MaybeResolved::Unresolved(unresolved) = item { - let table_fn = NormalResolver::new(self.resolver.tx, self.resolver.context) + let function = NormalResolver::new(self.resolver.tx, self.resolver.context) .require_resolve_table_function(&unresolved.reference)?; - let name = table_fn.name().to_string(); - let func = table_fn - .plan_and_initialize(self.resolver.context, unresolved.args.clone()) - .await?; + let resolved = match function.planner() { + TableFunctionPlanner::InOut(_) => { + ResolvedTableFunctionReference::InOut(function) + } + TableFunctionPlanner::Scan(planner) => { + let planned = planner + .plan( + self.resolver.context, + unresolved.args.positional.clone(), + unresolved.args.named.clone(), + ) + .await?; + + ResolvedTableFunctionReference::Scan(planned) + } + }; - // TODO: Marker indicating this needs to be executing remotely. - *item = MaybeResolved::Resolved( - ResolvedTableFunctionReference { name, func }, - LocationRequirement::Remote, - ) + *item = MaybeResolved::Resolved(resolved, LocationRequirement::Remote) } } diff --git a/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs b/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs index 28846230b..2dab392e7 100644 --- a/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs +++ b/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs @@ -1,40 +1,57 @@ -use rayexec_error::{OptionExt, Result}; +use std::collections::HashMap; + +use rayexec_bullet::scalar::OwnedScalarValue; +use rayexec_error::Result; use rayexec_parser::ast; use rayexec_proto::ProtoConv; use crate::database::DatabaseContext; -use crate::functions::table::inputs::TableFunctionInputs; -use crate::functions::table::PlannedTableFunction2; +use crate::functions::table::{PlannedTableFunction, TableFunction}; use crate::proto::DatabaseProtoConv; -/// A resolved table function reference. -#[derive(Debug, Clone, PartialEq)] -pub struct ResolvedTableFunctionReference { - /// Name of the original function. - /// - /// This is used to allow the user to reference the output of the function - /// if not provided an alias. - pub name: String, - /// The function. - pub func: Box, - // TODO: Maybe keep args here? +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct ConstantFunctionArgs { + pub positional: Vec, + pub named: HashMap, +} + +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum ResolvedTableFunctionReference { + /// Scan table functions can be fully resolved as their arguments are + /// constant. + Scan(PlannedTableFunction), + /// We have a function that's an in/out funciton, but we need to wait until + /// we're in the binding phase before planning this as it requires knowledge + /// of its inputs. + InOut(Box), +} + +impl ResolvedTableFunctionReference { + pub fn base_table_alias(&self) -> String { + match self { + ResolvedTableFunctionReference::Scan(func) => func.function.name().to_string(), + ResolvedTableFunctionReference::InOut(func) => func.name().to_string(), + } + } } impl DatabaseProtoConv for ResolvedTableFunctionReference { type ProtoType = rayexec_proto::generated::resolver::ResolvedTableFunctionReference; fn to_proto_ctx(&self, context: &DatabaseContext) -> Result { - Ok(Self::ProtoType { - name: self.name.clone(), - func: Some(self.func.to_proto_ctx(context)?), - }) + unimplemented!() + // Ok(Self::ProtoType { + // name: self.name.clone(), + // func: Some(self.func.to_proto_ctx(context)?), + // }) } fn from_proto_ctx(proto: Self::ProtoType, context: &DatabaseContext) -> Result { - Ok(Self { - name: proto.name, - func: DatabaseProtoConv::from_proto_ctx(proto.func.required("func")?, context)?, - }) + unimplemented!() + // Ok(Self { + // name: proto.name, + // func: DatabaseProtoConv::from_proto_ctx(proto.func.required("func")?, context)?, + // }) } } @@ -43,27 +60,33 @@ impl DatabaseProtoConv for ResolvedTableFunctionReference { pub struct UnresolvedTableFunctionReference { /// Original reference in the ast. pub reference: ast::ObjectReference, - /// Arguments to the function. + /// Constant arguments to the function. /// - /// Note that these are required to be constant and so we don't need to - /// delay binding. - pub args: TableFunctionInputs, + /// This currently assumes that any unresolved table function is for trying + /// to do a scan on a data source that's not registered (e.g. + /// `read_postgres` from wasm). + // TODO: Optionally set this? There's a possibility that the remote side has + // an in/out function that we don't know about, and so these args aren't + // actually needed. Not urgent to figure out right now. + pub args: ConstantFunctionArgs, } impl ProtoConv for UnresolvedTableFunctionReference { type ProtoType = rayexec_proto::generated::resolver::UnresolvedTableFunctionReference; fn to_proto(&self) -> Result { - Ok(Self::ProtoType { - reference: Some(self.reference.to_proto()?), - args: Some(self.args.to_proto()?), - }) + unimplemented!() + // Ok(Self::ProtoType { + // reference: Some(self.reference.to_proto()?), + // args: Some(self.args.to_proto()?), + // }) } fn from_proto(proto: Self::ProtoType) -> Result { - Ok(Self { - reference: ast::ObjectReference::from_proto(proto.reference.required("reference")?)?, - args: TableFunctionInputs::from_proto(proto.args.required("args")?)?, - }) + unimplemented!() + // Ok(Self { + // reference: ast::ObjectReference::from_proto(proto.reference.required("reference")?)?, + // args: TableFunctionInputs::from_proto(proto.args.required("args")?)?, + // }) } } diff --git a/crates/rayexec_parser/src/ast/from.rs b/crates/rayexec_parser/src/ast/from.rs index dadaa39c3..90d98f146 100644 --- a/crates/rayexec_parser/src/ast/from.rs +++ b/crates/rayexec_parser/src/ast/from.rs @@ -321,7 +321,7 @@ pub struct FromSubquery { pub struct FromTableFunction { pub lateral: bool, pub reference: T::TableFunctionReference, - pub args: T::TableFunctionArgs, + pub args: Vec>, } #[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] diff --git a/crates/rayexec_parser/src/meta.rs b/crates/rayexec_parser/src/meta.rs index 63cd72599..b71ead3b0 100644 --- a/crates/rayexec_parser/src/meta.rs +++ b/crates/rayexec_parser/src/meta.rs @@ -24,9 +24,6 @@ pub trait AstMeta: Clone { /// Reference to a table function. type TableFunctionReference: Debug + Clone + PartialEq + Serialize + DeserializeOwned; - /// Arguments to a table function. - type TableFunctionArgs: Debug + Clone + PartialEq + Serialize + DeserializeOwned; - /// Reference to a scalar or aggregate function. type FunctionReference: Debug + Clone + PartialEq + Serialize + DeserializeOwned; @@ -61,7 +58,6 @@ impl AstMeta for Raw { type ItemReference = ObjectReference; type TableReference = ObjectReference; type TableFunctionReference = ObjectReference; - type TableFunctionArgs = Vec>; type FunctionReference = ObjectReference; type SubqueryOptions = (); type DataType = DataType; From 6c2b986d16a916d663a91d57ccd0f634dab9cb79 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sat, 14 Dec 2024 15:25:49 -0600 Subject: [PATCH 11/23] logical planning --- .../logical/binder/bind_query/bind_from.rs | 137 +++++++++++++----- .../src/logical/logical_scan.rs | 8 +- .../src/logical/planner/plan_from.rs | 65 ++++++--- 3 files changed, 150 insertions(+), 60 deletions(-) diff --git a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs index ea9bb7b14..36851db2c 100644 --- a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs +++ b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs @@ -1,3 +1,4 @@ +use std::collections::HashMap; use std::sync::Arc; use rayexec_bullet::datatype::DataType; @@ -9,7 +10,7 @@ use crate::database::catalog_entry::CatalogEntry; use crate::expr::column_expr::ColumnExpr; use crate::expr::comparison_expr::{ComparisonExpr, ComparisonOperator}; use crate::expr::Expression; -use crate::functions::table::{PlannedTableFunction, PlannedTableFunction2}; +use crate::functions::table::{PlannedTableFunction, TableFunctionPlanner}; use crate::logical::binder::bind_context::{ BindContext, BindScopeRef, @@ -26,6 +27,8 @@ use crate::logical::resolver::resolve_context::ResolveContext; use crate::logical::resolver::resolved_table::ResolvedTableOrCteReference; use crate::logical::resolver::resolved_table_function::ResolvedTableFunctionReference; use crate::logical::resolver::{ResolvedMeta, ResolvedSubqueryOptions}; +use crate::optimizer::expr_rewrite::const_fold::ConstFold; +use crate::optimizer::expr_rewrite::ExpressionRewriteRule; #[derive(Debug, Clone, PartialEq, Eq)] pub struct BoundFrom { @@ -391,47 +394,111 @@ impl<'a> FromBinder<'a> { .table_functions .try_get_bound(function.reference)?; - match reference { - ResolvedTableFunctionReference::InOut(_) => { + let planned = match reference { + ResolvedTableFunctionReference::InOut(inout) => { // Handle in/out function planning now. We have everything we // need to plan its inputs. let expr_binder = BaseExpressionBinder::new(self.current, self.resolve_context); - unimplemented!() + + let mut positional = Vec::new(); + let mut named = HashMap::new(); + + for arg in function.args.iter() { + let recur = RecursionContext { + allow_aggregates: false, + allow_windows: false, + is_root: true, + }; + + match arg { + ast::FunctionArg::Unnamed { arg } => match arg { + ast::FunctionArgExpr::Expr(expr) => { + let expr = expr_binder.bind_expression( + bind_context, + expr, + &mut DefaultColumnBinder, + recur, + )?; + + positional.push(expr); + } + ast::FunctionArgExpr::Wildcard => { + return Err(RayexecError::new( + "Cannot plan a function with '*' as an argument", + )); + } + }, + ast::FunctionArg::Named { name, arg } => { + match arg { + ast::FunctionArgExpr::Expr(expr) => { + // Constants required. + let expr = expr_binder.bind_expression( + bind_context, + expr, + &mut DefaultColumnBinder, + recur, + )?; + + let val = + ConstFold::rewrite(bind_context.get_table_list(), expr)? + .try_into_scalar()?; + named.insert(name.as_normalized_string(), val); + } + ast::FunctionArgExpr::Wildcard => { + return Err(RayexecError::new( + "Cannot plan a function with '*' as an argument", + )); + } + } + } + } + } + + match inout.planner() { + TableFunctionPlanner::InOut(planner) => { + planner.plan(bind_context.get_table_list(), positional, named)? + } + TableFunctionPlanner::Scan(_) => { + return Err(RayexecError::new( + "Expected in/out planner, got scan planner", + )) + } + } } - ResolvedTableFunctionReference::Scan(planned) => { - // TODO: For table funcs that are reading files, it'd be nice to have - // the default alias be the base file path, not the function name. - let default_alias = TableAlias { - database: None, - schema: None, - table: reference.base_table_alias(), - }; + ResolvedTableFunctionReference::Scan(planned) => planned.clone(), + }; - let (names, types) = planned - .schema - .fields - .iter() - .map(|f| (f.name.clone(), f.datatype.clone())) - .unzip(); + // TODO: For table funcs that are reading files, it'd be nice to have + // the default alias be the base file path, not the function name. + let default_alias = TableAlias { + database: None, + schema: None, + table: reference.base_table_alias(), + }; - let table_ref = self.push_table_scope_with_from_alias( - bind_context, - Some(default_alias), - names, - types, - alias, - )?; + let (names, types) = planned + .schema + .fields + .iter() + .map(|f| (f.name.clone(), f.datatype.clone())) + .unzip(); - Ok(BoundFrom { - bind_ref: self.current, - item: BoundFromItem::TableFunction(BoundTableFunction { - table_ref, - location, - function: planned.clone(), - }), - }) - } - } + let table_ref = self.push_table_scope_with_from_alias( + bind_context, + Some(default_alias), + names, + types, + alias, + )?; + + Ok(BoundFrom { + bind_ref: self.current, + item: BoundFromItem::TableFunction(BoundTableFunction { + table_ref, + location, + function: planned, + }), + }) } fn bind_join( diff --git a/crates/rayexec_execution/src/logical/logical_scan.rs b/crates/rayexec_execution/src/logical/logical_scan.rs index be01af108..56f1e2e46 100644 --- a/crates/rayexec_execution/src/logical/logical_scan.rs +++ b/crates/rayexec_execution/src/logical/logical_scan.rs @@ -11,7 +11,7 @@ use super::statistics::StatisticsValue; use crate::database::catalog_entry::CatalogEntry; use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; use crate::expr::Expression; -use crate::functions::table::PlannedTableFunction2; +use crate::functions::table::PlannedTableFunction; // TODO: Probably remove view from this. // Maybe just split it all up. @@ -23,7 +23,7 @@ pub enum ScanSource { source: Arc, }, TableFunction { - function: Box, + function: PlannedTableFunction, }, ExpressionList { rows: Vec>, @@ -39,7 +39,7 @@ impl ScanSource { pub fn cardinality(&self) -> StatisticsValue { match self { Self::Table { .. } => StatisticsValue::Unknown, - Self::TableFunction { function } => function.cardinality(), + Self::TableFunction { function } => function.cardinality, Self::ExpressionList { rows } => StatisticsValue::Exact(rows.len()), Self::View { .. } => StatisticsValue::Unknown, } @@ -95,7 +95,7 @@ impl Explainable for LogicalScan { source, } => ent = ent.with_value("source", format!("{catalog}.{schema}.{}", source.name)), ScanSource::TableFunction { function } => { - ent = ent.with_value("function_name", function.table_function().name()) + ent = ent.with_value("function_name", function.function.name()) } ScanSource::ExpressionList { rows } => { ent = ent.with_value("num_rows", rows.len()); diff --git a/crates/rayexec_execution/src/logical/planner/plan_from.rs b/crates/rayexec_execution/src/logical/planner/plan_from.rs index a692c6a66..bb45e5221 100644 --- a/crates/rayexec_execution/src/logical/planner/plan_from.rs +++ b/crates/rayexec_execution/src/logical/planner/plan_from.rs @@ -7,10 +7,12 @@ use crate::expr::column_expr::ColumnExpr; use crate::expr::comparison_expr::ComparisonExpr; use crate::expr::literal_expr::LiteralExpr; use crate::expr::{self, Expression}; +use crate::functions::table::TableFunctionImpl; use crate::logical::binder::bind_context::BindContext; use crate::logical::binder::bind_query::bind_from::{BoundFrom, BoundFromItem, BoundJoin}; use crate::logical::logical_empty::LogicalEmpty; use crate::logical::logical_filter::LogicalFilter; +use crate::logical::logical_inout::LogicalInOut; use crate::logical::logical_join::{ ComparisonCondition, JoinType, @@ -72,27 +74,48 @@ impl FromPlanner { names.extend(table.column_names.iter().cloned()); } - let projection = (0..types.len()).collect(); - - let source = ScanSource::TableFunction { - function: func.function, - }; - let estimated_cardinality = source.cardinality(); - - Ok(LogicalOperator::Scan(Node { - node: LogicalScan { - table_ref: func.table_ref, - types, - names, - projection, - did_prune_columns: false, - scan_filters: Vec::new(), - source, - }, - location: func.location, - children: Vec::new(), - estimated_cardinality, - })) + match &func.function.function_impl { + TableFunctionImpl::Scan(_) => { + let projection = (0..types.len()).collect(); + + let source = ScanSource::TableFunction { + function: func.function, + }; + let estimated_cardinality = source.cardinality(); + + Ok(LogicalOperator::Scan(Node { + node: LogicalScan { + table_ref: func.table_ref, + types, + names, + projection, + did_prune_columns: false, + scan_filters: Vec::new(), + source, + }, + location: func.location, + children: Vec::new(), + estimated_cardinality, + })) + } + TableFunctionImpl::InOut(_) => { + let cardinality = func.function.cardinality; + + // In/out always requires one input. Initialize its + // input with an empty operator. Subquery planning will + // take care of the lateral binding and changing its + // child as needed. + Ok(LogicalOperator::InOut(Node { + node: LogicalInOut { + table_ref: func.table_ref, + function: func.function, + }, + location: func.location, + children: vec![LogicalOperator::EMPTY], + estimated_cardinality: cardinality, + })) + } + } } BoundFromItem::Subquery(subquery) => { let plan = QueryPlanner.plan(bind_context, *subquery.subquery)?; From da25b7c8d53287fb5cd25e2bc6725eaa335fb8cf Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 10:50:56 -0600 Subject: [PATCH 12/23] impl for generate series --- .../src/execution/operators/empty.rs | 10 +- .../src/execution/operators/table_function.rs | 67 +++--- .../src/functions/table/builtin/series.rs | 223 +++++------------- .../src/functions/table/scan.rs | 41 ++++ 4 files changed, 141 insertions(+), 200 deletions(-) diff --git a/crates/rayexec_execution/src/execution/operators/empty.rs b/crates/rayexec_execution/src/execution/operators/empty.rs index 01e610c58..aa63aee64 100644 --- a/crates/rayexec_execution/src/execution/operators/empty.rs +++ b/crates/rayexec_execution/src/execution/operators/empty.rs @@ -23,17 +23,11 @@ pub struct EmptyPartitionState { finished: bool, } +/// A dummy operator that produces a single batch containing no columns and a +/// single row for each partition. #[derive(Debug)] pub struct PhysicalEmpty; -impl PhysicalEmpty { - pub fn create_states(num_partitions: usize) -> Vec { - (0..num_partitions) - .map(|_| EmptyPartitionState { finished: false }) - .collect() - } -} - impl ExecutableOperator for PhysicalEmpty { fn create_states( &self, diff --git a/crates/rayexec_execution/src/execution/operators/table_function.rs b/crates/rayexec_execution/src/execution/operators/table_function.rs index ddc5f1cdf..fcd4c9444 100644 --- a/crates/rayexec_execution/src/execution/operators/table_function.rs +++ b/crates/rayexec_execution/src/execution/operators/table_function.rs @@ -5,8 +5,7 @@ use std::task::{Context, Poll}; use futures::future::BoxFuture; use futures::FutureExt; use rayexec_bullet::batch::Batch; -use rayexec_error::{OptionExt, RayexecError, Result}; -use rayexec_proto::ProtoConv; +use rayexec_error::{RayexecError, Result}; use super::util::futures::make_static; use super::{ @@ -21,12 +20,13 @@ use super::{ }; use crate::database::DatabaseContext; use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; -use crate::functions::table::PlannedTableFunction2; +use crate::functions::table::scan::TableScanState; +use crate::functions::table::{PlannedTableFunction, TableFunctionImpl}; use crate::proto::DatabaseProtoConv; -use crate::storage::table_storage::{DataTableScan, Projections}; +use crate::storage::table_storage::Projections; pub struct TableFunctionPartitionState { - scan: Box, + scan_state: Box, /// In progress pull we're working on. future: Option>>>, } @@ -40,12 +40,12 @@ impl fmt::Debug for TableFunctionPartitionState { #[derive(Debug)] pub struct PhysicalTableFunction { - function: Box, + function: PlannedTableFunction, projections: Projections, } impl PhysicalTableFunction { - pub fn new(function: Box, projections: Projections) -> Self { + pub fn new(function: PlannedTableFunction, projections: Projections) -> Self { PhysicalTableFunction { function, projections, @@ -59,15 +59,26 @@ impl ExecutableOperator for PhysicalTableFunction { _context: &DatabaseContext, partitions: Vec, ) -> Result { - let data_table = self.function.datatable()?; + let scan_func = match &self.function.function_impl { + TableFunctionImpl::Scan(scan) => scan, + _ => { + return Err(RayexecError::new(format!( + "Table function '{}' not a scan function", + self.function.function.name() + ))) + } + }; - // TODO: Pushdown projections, filters - let scans = data_table.scan(self.projections.clone(), partitions[0])?; + // TODO: Pushdown filters + let scans = scan_func.scan(self.projections.clone(), partitions[0])?; let states = scans .into_iter() - .map(|scan| { - PartitionState::TableFunction(TableFunctionPartitionState { scan, future: None }) + .map(|scan_state| { + PartitionState::TableFunction(TableFunctionPartitionState { + scan_state, + future: None, + }) }) .collect(); @@ -119,7 +130,7 @@ impl ExecutableOperator for PhysicalTableFunction { } } - let mut future = state.scan.pull(); + let mut future = state.scan_state.pull(); match future.poll_unpin(cx) { Poll::Ready(Ok(Some(batch))) => Ok(PollPull::Computed(batch.into())), Poll::Ready(Ok(None)) => Ok(PollPull::Exhausted), @@ -146,21 +157,23 @@ impl Explainable for PhysicalTableFunction { impl DatabaseProtoConv for PhysicalTableFunction { type ProtoType = rayexec_proto::generated::execution::PhysicalTableFunction; - fn to_proto_ctx(&self, context: &DatabaseContext) -> Result { - Ok(Self::ProtoType { - function: Some(self.function.to_proto_ctx(context)?), - projections: Some(self.projections.to_proto()?), - }) + fn to_proto_ctx(&self, _context: &DatabaseContext) -> Result { + unimplemented!() + // Ok(Self::ProtoType { + // function: Some(self.function.to_proto_ctx(context)?), + // projections: Some(self.projections.to_proto()?), + // }) } - fn from_proto_ctx(proto: Self::ProtoType, context: &DatabaseContext) -> Result { - // TODO: https://github.com/GlareDB/rayexec/issues/278 - Ok(Self { - function: DatabaseProtoConv::from_proto_ctx( - proto.function.required("function")?, - context, - )?, - projections: ProtoConv::from_proto(proto.projections.required("projections")?)?, - }) + fn from_proto_ctx(_proto: Self::ProtoType, _context: &DatabaseContext) -> Result { + unimplemented!() + // // TODO: https://github.com/GlareDB/rayexec/issues/278 + // Ok(Self { + // function: DatabaseProtoConv::from_proto_ctx( + // proto.function.required("function")?, + // context, + // )?, + // projections: ProtoConv::from_proto(proto.projections.required("projections")?)?, + // }) } } diff --git a/crates/rayexec_execution/src/functions/table/builtin/series.rs b/crates/rayexec_execution/src/functions/table/builtin/series.rs index 52ecf4170..54c66d01b 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/series.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/series.rs @@ -1,30 +1,34 @@ -use std::sync::Arc; +use std::collections::HashMap; use std::task::{Context, Waker}; -use futures::future::BoxFuture; -use rayexec_bullet::array::{Array, ArrayData}; +use rayexec_bullet::array::Array; use rayexec_bullet::batch::Batch; use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_bullet::executor::physical_type::PhysicalI64; use rayexec_bullet::executor::scalar::UnaryExecutor; use rayexec_bullet::field::{Field, Schema}; +use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_bullet::storage::PrimitiveStorage; use rayexec_error::{RayexecError, Result}; -use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; -use serde::{Deserialize, Serialize}; -use crate::database::DatabaseContext; use crate::execution::operators::{PollFinalize, PollPull, PollPush}; +use crate::expr::{self, Expression}; use crate::functions::table::inout::{TableInOutFunction, TableInOutPartitionState}; -use crate::functions::table::{PlannedTableFunction2, TableFunction, TableFunctionInputs}; -use crate::functions::{FunctionInfo, Signature}; -use crate::storage::table_storage::{ - DataTable, - DataTableScan, - EmptyTableScan, - ProjectedScan, - Projections, +use crate::functions::table::{ + InOutPlanner, + PlannedTableFunction, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, }; +use crate::functions::{ + invalid_input_types_error, + plan_check_num_args_one_of, + FunctionInfo, + Signature, +}; +use crate::logical::binder::table_list::TableList; +use crate::logical::statistics::StatisticsValue; #[derive(Debug, Clone, Copy, PartialEq, Eq)] pub struct GenerateSeries; @@ -51,166 +55,53 @@ impl FunctionInfo for GenerateSeries { } impl TableFunction for GenerateSeries { - fn plan_and_initialize<'a>( - &self, - _context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - Box::pin(async move { Self::plan_and_initialize_inner(args) }) - } - - fn decode_state(&self, state: &[u8]) -> Result> { - let mut packed = PackedDecoder::new(state); - let start = packed.decode_next()?; - let stop = packed.decode_next()?; - let step = packed.decode_next()?; - Ok(Box::new(GenerateSeriesI64 { start, stop, step })) - } -} - -impl GenerateSeries { - fn plan_and_initialize_inner( - args: TableFunctionInputs, - ) -> Result> { - if !args.named.is_empty() { - return Err(RayexecError::new( - "generate_series does not accept named arguments", - )); - } - - let mut args = args.clone(); - let [start, stop, step] = match args.positional.len() { - 2 => { - let stop = args.positional.pop().unwrap().try_as_i64()?; - let start = args.positional.pop().unwrap().try_as_i64()?; - [start, stop, 1] - } - 3 => { - let step = args.positional.pop().unwrap().try_as_i64()?; - let stop = args.positional.pop().unwrap().try_as_i64()?; - let start = args.positional.pop().unwrap().try_as_i64()?; - [start, stop, step] - } - _ => { - return Err(RayexecError::new( - "generate_series requires 2 or 3 arguments", - )); - } - }; - - if step == 0 { - return Err(RayexecError::new("'step' may not be zero")); - } - - Ok(Box::new(GenerateSeriesI64 { start, stop, step }) as _) + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::InOut(&GenerateSeriesInOutPlanner) } } -#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] -pub struct GenerateSeriesI64 { - start: i64, - stop: i64, - step: i64, -} - -impl PlannedTableFunction2 for GenerateSeriesI64 { - fn encode_state(&self, state: &mut Vec) -> Result<()> { - let mut packed = PackedEncoder::new(state); - packed.encode_next(&self.start)?; - packed.encode_next(&self.stop)?; - packed.encode_next(&self.step)?; - Ok(()) - } - - fn table_function(&self) -> &dyn TableFunction { - &GenerateSeries - } - - fn schema(&self) -> Schema { - Schema::new([Field::new("generate_series", DataType::Int64, false)]) - } - - fn datatable(&self) -> Result> { - Ok(Box::new(self.clone())) - } -} +#[derive(Debug, Clone)] +pub struct GenerateSeriesInOutPlanner; -impl DataTable for GenerateSeriesI64 { - fn scan( +impl InOutPlanner for GenerateSeriesInOutPlanner { + fn plan( &self, - projections: Projections, - num_partitions: usize, - ) -> Result>> { - let mut scans: Vec> = vec![Box::new(ProjectedScan::new( - GenerateSeriesScan { - batch_size: 1024, - exhausted: false, - curr: self.start, - stop: self.stop, - step: self.step, - }, - projections, - ))]; - scans.extend((1..num_partitions).map(|_| Box::new(EmptyTableScan) as _)); - - Ok(scans) - } -} - -#[derive(Debug, PartialEq)] -struct GenerateSeriesScan { - batch_size: usize, - exhausted: bool, - curr: i64, - stop: i64, - step: i64, -} - -impl GenerateSeriesScan { - fn generate_next(&mut self) -> Option { - if self.exhausted { - return None; + table_list: &TableList, + mut positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { + plan_check_num_args_one_of(&GenerateSeries, &positional_inputs, [2, 3])?; + if !named_inputs.is_empty() { + return Err(RayexecError::new(format!( + "'{}' does not accept named arguments", + GenerateSeries.name() + ))); } - let mut series: Vec<_> = Vec::new(); - if self.curr < self.stop && self.step > 0 { - // Going up. - let mut count = 0; - while self.curr <= self.stop && count < self.batch_size { - series.push(self.curr); - self.curr += self.step; - count += 1; - } - } else if self.curr > self.stop && self.step < 0 { - // Going down. - let mut count = 0; - while self.curr >= self.stop && count < self.batch_size { - series.push(self.curr); - self.curr += self.step; - count += 1; - } - } + let datatypes = positional_inputs + .iter() + .map(|expr| expr.datatype(table_list)) + .collect::>>()?; - if series.len() < self.batch_size { - self.exhausted = true; + for datatype in &datatypes { + if datatype != &DataType::Int64 { + return Err(invalid_input_types_error(&GenerateSeries, &datatypes)); + } } - // Calculate the start value for the next iteration. - if let Some(last) = series.last() { - self.curr = *last + self.step; + if positional_inputs.len() == 2 { + // Add constant for the 'step' argument. + positional_inputs.push(expr::lit(1 as i64)) } - let col = - Array::new_with_array_data(DataType::Int64, ArrayData::Int64(Arc::new(series.into()))); - let batch = Batch::try_new([col]).expect("batch to be valid"); - - Some(batch) - } -} - -impl DataTableScan for GenerateSeriesScan { - fn pull(&mut self) -> BoxFuture<'_, Result>> { - Box::pin(async { Ok(self.generate_next()) }) + Ok(PlannedTableFunction { + function: Box::new(GenerateSeries), + positional_inputs, + named_inputs, + function_impl: TableFunctionImpl::InOut(Box::new(GenerateSeriesInOutImpl)), + cardinality: StatisticsValue::Unknown, + schema: Schema::new([Field::new("generate_series", DataType::Int64, false)]), + }) } } @@ -361,17 +252,19 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { UnaryExecutor::value_at::(batch.column(0).unwrap(), self.row_idx)?; let end = UnaryExecutor::value_at::(batch.column(1).unwrap(), self.row_idx)?; + let step = + UnaryExecutor::value_at::(batch.column(2).unwrap(), self.row_idx)?; // Use values from start/end if they're both not null. Otherwise use // parameters that produce an empty array. - match (start, end) { - (Some(start), Some(end)) => { + match (start, end, step) { + (Some(start), Some(end), Some(step)) => { self.params = SeriesParams { batch_size: self.batch_size, exhausted: false, curr: start, stop: end, - step: 1, // TODO + step, } } _ => { diff --git a/crates/rayexec_execution/src/functions/table/scan.rs b/crates/rayexec_execution/src/functions/table/scan.rs index b4bcea735..122cf5bf5 100644 --- a/crates/rayexec_execution/src/functions/table/scan.rs +++ b/crates/rayexec_execution/src/functions/table/scan.rs @@ -24,3 +24,44 @@ impl Clone for Box { dyn_clone::clone_box(&**self) } } + +/// Helper for wrapping an unprojected scan with a projections list to produce +/// projected batches. +/// +/// This is inefficient compared to handling the projection in the scan itself +/// since this projects a batch after it's already been read. +#[derive(Debug)] +pub struct ProjectedTableScanState { + pub projections: Projections, + pub scan_state: S, +} + +impl ProjectedTableScanState { + pub fn new(scan_state: S, projections: Projections) -> Self { + ProjectedTableScanState { + projections, + scan_state, + } + } + + async fn pull_inner(&mut self) -> Result> { + let batch = match self.scan_state.pull().await? { + Some(batch) => batch, + None => return Ok(None), + }; + + match self.projections.column_indices.as_ref() { + Some(indices) => { + let batch = batch.project(indices); + Ok(Some(batch)) + } + None => Ok(Some(batch)), + } + } +} + +impl TableScanState for ProjectedTableScanState { + fn pull(&mut self) -> BoxFuture<'_, Result>> { + Box::pin(async { self.pull_inner().await }) + } +} From ae7e2e3c54bd68d42558d238a8b1f12c2bc728ac Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 13:00:31 -0600 Subject: [PATCH 13/23] begin handling correlated columns --- .../src/execution/operators/table_inout.rs | 13 ++- crates/rayexec_execution/src/functions/mod.rs | 1 + .../src/functions/scalar/mod.rs | 1 + .../src/functions/table/builtin/series.rs | 103 ++++++++++-------- .../src/functions/table/inout.rs | 2 +- .../logical/binder/bind_query/bind_from.rs | 8 ++ .../src/logical/binder/expr_binder.rs | 54 +++++++++ .../src/logical/logical_inout.rs | 12 +- .../src/logical/planner/plan_subquery.rs | 16 +++ .../functions/table/generate_series.slt | 6 + 10 files changed, 164 insertions(+), 52 deletions(-) diff --git a/crates/rayexec_execution/src/execution/operators/table_inout.rs b/crates/rayexec_execution/src/execution/operators/table_inout.rs index d70757334..f5cc3fc7c 100644 --- a/crates/rayexec_execution/src/execution/operators/table_inout.rs +++ b/crates/rayexec_execution/src/execution/operators/table_inout.rs @@ -79,7 +79,18 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid partition state: {other:?}"), }; - state.function_state.poll_push(cx, batch) + let inputs = self + .function_inputs + .iter() + .map(|function| { + let arr = function.eval(&batch)?; + Ok(arr.into_owned()) + }) + .collect::>>()?; + + let inputs = Batch::try_new(inputs)?; + + state.function_state.poll_push(cx, inputs) } fn poll_finalize_push( diff --git a/crates/rayexec_execution/src/functions/mod.rs b/crates/rayexec_execution/src/functions/mod.rs index 83b4fe22d..a2a6941ea 100644 --- a/crates/rayexec_execution/src/functions/mod.rs +++ b/crates/rayexec_execution/src/functions/mod.rs @@ -12,6 +12,7 @@ use fmtutil::IntoDisplayableSlice; use implicit::{implicit_cast_score, NO_CAST_SCORE}; use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_error::{RayexecError, Result}; +use scalar::ScalarFunction; /// Function signature. // TODO: Include named args. diff --git a/crates/rayexec_execution/src/functions/scalar/mod.rs b/crates/rayexec_execution/src/functions/scalar/mod.rs index 94251627c..e53158374 100644 --- a/crates/rayexec_execution/src/functions/scalar/mod.rs +++ b/crates/rayexec_execution/src/functions/scalar/mod.rs @@ -1,5 +1,6 @@ pub mod builtin; +use std::any::Any; use std::fmt::Debug; use std::hash::Hash; diff --git a/crates/rayexec_execution/src/functions/table/builtin/series.rs b/crates/rayexec_execution/src/functions/table/builtin/series.rs index 54c66d01b..e2c0c4196 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/series.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/series.rs @@ -118,9 +118,14 @@ impl TableInOutFunction for GenerateSeriesInOutImpl { Box::new(GenerateSeriesInOutPartitionState { batch_size: 1024, // TODO batch: None, - row_idx: 0, + next_row_idx: 0, finished: false, - params: SeriesParams::default(), + params: SeriesParams { + exhausted: true, // Triggers param update on first pull + curr: 0, + stop: 0, + step: 0, + }, push_waker: None, pull_waker: None, }) as _ @@ -131,9 +136,8 @@ impl TableInOutFunction for GenerateSeriesInOutImpl { } } -#[derive(Debug, Clone, Default)] +#[derive(Debug, Clone)] struct SeriesParams { - batch_size: usize, exhausted: bool, curr: i64, @@ -143,14 +147,14 @@ struct SeriesParams { impl SeriesParams { /// Generate the next set of rows using the current parameters. - fn generate_next(&mut self) -> Array { + fn generate_next(&mut self, batch_size: usize) -> Array { debug_assert!(!self.exhausted); let mut series: Vec = Vec::new(); if self.curr < self.stop && self.step > 0 { // Going up. let mut count = 0; - while self.curr <= self.stop && count < self.batch_size { + while self.curr <= self.stop && count < batch_size { series.push(self.curr); self.curr += self.step; count += 1; @@ -158,14 +162,14 @@ impl SeriesParams { } else if self.curr > self.stop && self.step < 0 { // Going down. let mut count = 0; - while self.curr >= self.stop && count < self.batch_size { + while self.curr >= self.stop && count < batch_size { series.push(self.curr); self.curr += self.step; count += 1; } } - if series.len() < self.batch_size { + if series.len() < batch_size { self.exhausted = true; } @@ -183,8 +187,8 @@ pub struct GenerateSeriesInOutPartitionState { batch_size: usize, /// Batch we're working on. batch: Option, - /// Row index we're on. - row_idx: usize, + /// Row index we should try next. + next_row_idx: usize, /// If we're finished. finished: bool, /// Current params. @@ -205,62 +209,61 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { } self.batch = Some(batch); - self.row_idx = 0; + self.next_row_idx = 0; Ok(PollPush::Pushed) } fn poll_finalize_push(&mut self, _cx: &mut Context) -> Result { self.finished = true; + if let Some(waker) = self.pull_waker.take() { + waker.wake(); + } + Ok(PollFinalize::Finalized) } fn poll_pull(&mut self, cx: &mut Context) -> Result { - let batch = match &self.batch { - Some(batch) => batch, - None => { - if self.finished { - return Ok(PollPull::Exhausted); - } - - // No batch to work on, come back later. - self.pull_waker = Some(cx.waker().clone()); - if let Some(push_waker) = self.push_waker.take() { - push_waker.wake() - } - return Ok(PollPull::Pending); - } - }; - if self.params.exhausted { - // Move to next row to process. - self.row_idx += 1; + let batch = match &self.batch { + Some(batch) => batch, + None => { + if self.finished { + return Ok(PollPull::Exhausted); + } - if self.row_idx >= batch.num_rows() { - // Need more input. - self.batch = None; - self.pull_waker = Some(cx.waker().clone()); - if let Some(push_waker) = self.push_waker.take() { - push_waker.wake() + // No batch to work on, come back later. + self.pull_waker = Some(cx.waker().clone()); + if let Some(push_waker) = self.push_waker.take() { + push_waker.wake() + } + return Ok(PollPull::Pending); } - - return Ok(PollPull::Pending); - } + }; // Generate new params from row. - let start = - UnaryExecutor::value_at::(batch.column(0).unwrap(), self.row_idx)?; - let end = - UnaryExecutor::value_at::(batch.column(1).unwrap(), self.row_idx)?; - let step = - UnaryExecutor::value_at::(batch.column(2).unwrap(), self.row_idx)?; + let start = UnaryExecutor::value_at::( + batch.column(0).unwrap(), + self.next_row_idx, + )?; + let end = UnaryExecutor::value_at::( + batch.column(1).unwrap(), + self.next_row_idx, + )?; + let step = UnaryExecutor::value_at::( + batch.column(2).unwrap(), + self.next_row_idx, + )?; // Use values from start/end if they're both not null. Otherwise use // parameters that produce an empty array. match (start, end, step) { (Some(start), Some(end), Some(step)) => { + if step == 0 { + return Err(RayexecError::new("'step' may not be zero")); + } + self.params = SeriesParams { - batch_size: self.batch_size, exhausted: false, curr: start, stop: end, @@ -269,7 +272,6 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { } _ => { self.params = SeriesParams { - batch_size: self.batch_size, exhausted: false, curr: 1, stop: 0, @@ -278,10 +280,15 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { } } - // TODO: Validate params. + // Increment next row to use when current row exhausted. + self.next_row_idx += 1; + if self.next_row_idx >= batch.num_rows() { + // Need more input. + self.batch = None; + } } - let out = self.params.generate_next(); + let out = self.params.generate_next(self.batch_size); let batch = Batch::try_new([out])?; Ok(PollPull::Computed(batch.into())) diff --git a/crates/rayexec_execution/src/functions/table/inout.rs b/crates/rayexec_execution/src/functions/table/inout.rs index 4ca71744a..225a7686c 100644 --- a/crates/rayexec_execution/src/functions/table/inout.rs +++ b/crates/rayexec_execution/src/functions/table/inout.rs @@ -15,7 +15,7 @@ pub trait TableInOutFunction: Debug + Sync + Send + DynClone { } pub trait TableInOutPartitionState: Debug + Sync + Send { - fn poll_push(&mut self, cx: &mut Context, batch: Batch) -> Result; + fn poll_push(&mut self, cx: &mut Context, inputs: Batch) -> Result; fn poll_finalize_push(&mut self, cx: &mut Context) -> Result; fn poll_pull(&mut self, cx: &mut Context) -> Result; } diff --git a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs index 36851db2c..b17a78a8d 100644 --- a/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs +++ b/crates/rayexec_execution/src/logical/binder/bind_query/bind_from.rs @@ -454,6 +454,14 @@ impl<'a> FromBinder<'a> { } } + // Note only positional input casts for now. Signatures don't + // have a notion of named arguments yet. + let positional = expr_binder.apply_casts_for_table_function( + bind_context, + inout.as_ref(), + positional, + )?; + match inout.planner() { TableFunctionPlanner::InOut(planner) => { planner.plan(bind_context.get_table_list(), positional, named)? diff --git a/crates/rayexec_execution/src/logical/binder/expr_binder.rs b/crates/rayexec_execution/src/logical/binder/expr_binder.rs index 148360de0..9fc0cf5d9 100644 --- a/crates/rayexec_execution/src/logical/binder/expr_binder.rs +++ b/crates/rayexec_execution/src/logical/binder/expr_binder.rs @@ -27,6 +27,7 @@ use crate::functions::scalar::builtin::is; use crate::functions::scalar::builtin::list::{ListExtract, ListValues}; use crate::functions::scalar::builtin::string::{Concat, Like, StartsWith, Substring}; use crate::functions::scalar::ScalarFunction; +use crate::functions::table::TableFunction; use crate::functions::CastType; use crate::logical::binder::bind_query::bind_modifier::BoundOrderByExpr; use crate::logical::binder::bind_query::QueryBinder; @@ -1517,4 +1518,57 @@ impl<'a> BaseExpressionBinder<'a> { Ok(inputs) } } + + // TODO: Reduce duplication with scalar one. + // + // Upcasting to FunctionInfo would be cool. + // See: + pub(crate) fn apply_casts_for_table_function( + &self, + bind_context: &BindContext, + table: &dyn TableFunction, + inputs: Vec, + ) -> Result> { + let input_datatypes = inputs + .iter() + .map(|expr| expr.datatype(bind_context.get_table_list())) + .collect::>>()?; + + if table.exact_signature(&input_datatypes).is_some() { + // Exact + Ok(inputs) + } else { + // Try to find candidates that we can cast to. + let mut candidates = table.candidate(&input_datatypes); + + if candidates.is_empty() { + // TODO: Better error. + return Err(RayexecError::new(format!( + "Invalid inputs to '{}': {}", + table.name(), + input_datatypes.display_with_brackets(), + ))); + } + + // TODO: Maybe more sophisticated candidate selection. + let candidate = candidates.swap_remove(0); + + // Apply casts where needed. + let inputs = inputs + .into_iter() + .zip(candidate.casts) + .map(|(input, cast_to)| { + Ok(match cast_to { + CastType::Cast { to, .. } => Expression::Cast(CastExpr { + to: DataType::try_default_datatype(to)?, + expr: Box::new(input), + }), + CastType::NoCastNeeded => input, + }) + }) + .collect::>>()?; + + Ok(inputs) + } + } } diff --git a/crates/rayexec_execution/src/logical/logical_inout.rs b/crates/rayexec_execution/src/logical/logical_inout.rs index 18c06fd07..88fb8edf0 100644 --- a/crates/rayexec_execution/src/logical/logical_inout.rs +++ b/crates/rayexec_execution/src/logical/logical_inout.rs @@ -17,8 +17,16 @@ pub struct LogicalInOut { } impl Explainable for LogicalInOut { - fn explain_entry(&self, _conf: ExplainConfig) -> ExplainEntry { - ExplainEntry::new("TableInOut").with_value("function", self.function.function.name()) + fn explain_entry(&self, conf: ExplainConfig) -> ExplainEntry { + let mut ent = ExplainEntry::new("TableInOut") + .with_value("function", self.function.function.name()) + .with_values_context("inputs", conf, &self.function.positional_inputs); + + if conf.verbose { + ent = ent.with_value("table_ref", self.table_ref); + } + + ent } } diff --git a/crates/rayexec_execution/src/logical/planner/plan_subquery.rs b/crates/rayexec_execution/src/logical/planner/plan_subquery.rs index f49b85233..00c942494 100644 --- a/crates/rayexec_execution/src/logical/planner/plan_subquery.rs +++ b/crates/rayexec_execution/src/logical/planner/plan_subquery.rs @@ -57,6 +57,7 @@ impl SubqueryPlanner { mut conditions: Vec, lateral_columns: Vec, ) -> Result { + println!("PLANNING LATERAL"); // Very similar to planning correlated subqueries (becuase it is), just // we already have the correlated columns we're flattening for. @@ -653,6 +654,11 @@ impl DependentJoinPushdown { self.any_expression_has_correlation(order.node.exprs.iter().map(|e| &e.expr)); has_correlation |= self.find_correlations_in_children(&order.children)?; } + LogicalOperator::InOut(inout) => { + has_correlation = + self.any_expression_has_correlation(&inout.node.function.positional_inputs); + has_correlation |= self.find_correlations_in_children(&inout.children)?; + } _ => (), } @@ -796,6 +802,15 @@ impl DependentJoinPushdown { Ok(()) } + LogicalOperator::InOut(inout) => { + self.pushdown_children(bind_context, &mut inout.children)?; + self.rewrite_expressions(&mut inout.node.function.positional_inputs)?; + + // TODO: Need to check if we should propagate columns through + // the inout. If we do, it should just be a cross join. + + Ok(()) + } LogicalOperator::Filter(filter) => { self.pushdown_children(bind_context, &mut filter.children)?; self.rewrite_expression(&mut filter.node.filter)?; @@ -896,6 +911,7 @@ impl DependentJoinPushdown { Ok(()) } + // TODO: Should accept logical node trait. fn any_expression_has_correlation<'a>( &self, exprs: impl IntoIterator, diff --git a/slt/standard/functions/table/generate_series.slt b/slt/standard/functions/table/generate_series.slt index b94624526..85956682b 100644 --- a/slt/standard/functions/table/generate_series.slt +++ b/slt/standard/functions/table/generate_series.slt @@ -98,3 +98,9 @@ select i * 2 from (select * from generate_series(1, 5)) as t(i); 8 10 +# Lateral input + +query II +select * from (values (3), (4)) v(a), generate_series(1, a) order by 1,2; +---- + From e796537180aaacf469abf9de7176bd3ef73eb7de Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 13:57:55 -0600 Subject: [PATCH 14/23] working lateral --- .../intermediate/planner/plan_inout.rs | 8 +- .../src/execution/operators/table_inout.rs | 73 ++++++++++++++++++- .../src/functions/table/builtin/series.rs | 20 +++-- .../src/functions/table/inout.rs | 11 ++- .../src/logical/logical_inout.rs | 32 +++++++- .../src/logical/planner/plan_from.rs | 4 +- .../src/logical/planner/plan_subquery.rs | 39 +++++++++- .../functions/table/generate_series.slt | 28 +++++++ 8 files changed, 195 insertions(+), 20 deletions(-) diff --git a/crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs b/crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs index 7b9bfe3e5..2c7d1926b 100644 --- a/crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs +++ b/crates/rayexec_execution/src/execution/intermediate/planner/plan_inout.rs @@ -23,12 +23,18 @@ impl IntermediatePipelineBuildState<'_> { let function_inputs = self .expr_planner .plan_scalars(&input_refs, &inout.node.function.positional_inputs) - .context("Failed to plan expressions for table inout")?; + .context("Failed to plan input expressions for table inout")?; + + let projected_outputs = self + .expr_planner + .plan_scalars(&input_refs, &inout.node.projected_outputs) + .context("Failed to plan additional output expressions for table inout")?; let operator = IntermediateOperator { operator: Arc::new(PhysicalOperator::TableInOut(PhysicalTableInOut { function: inout.node.function, function_inputs, + projected_outputs, })), partitioning_requirement: None, }; diff --git a/crates/rayexec_execution/src/execution/operators/table_inout.rs b/crates/rayexec_execution/src/execution/operators/table_inout.rs index f5cc3fc7c..bdd735d04 100644 --- a/crates/rayexec_execution/src/execution/operators/table_inout.rs +++ b/crates/rayexec_execution/src/execution/operators/table_inout.rs @@ -1,7 +1,9 @@ use std::sync::Arc; use std::task::Context; +use rayexec_bullet::array::Array; use rayexec_bullet::batch::Batch; +use rayexec_bullet::selection::SelectionVector; use rayexec_error::{RayexecError, Result}; use super::{ @@ -22,6 +24,8 @@ use crate::functions::table::{inout, PlannedTableFunction, TableFunctionImpl}; #[derive(Debug)] pub struct TableInOutPartitionState { function_state: Box, + /// Additional outputs that will be included on the output batch. + additional_outputs: Vec, } #[derive(Debug)] @@ -30,6 +34,8 @@ pub struct PhysicalTableInOut { pub function: PlannedTableFunction, /// Input expressions to the table function. pub function_inputs: Vec, + /// Output projections. + pub projected_outputs: Vec, } impl ExecutableOperator for PhysicalTableInOut { @@ -55,6 +61,7 @@ impl ExecutableOperator for PhysicalTableInOut { .map(|state| { PartitionState::TableInOut(TableInOutPartitionState { function_state: state, + additional_outputs: Vec::new(), }) }) .collect(); @@ -79,18 +86,48 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid partition state: {other:?}"), }; + // TODO: Don't do this. + let orig = batch.clone(); + let inputs = self .function_inputs .iter() - .map(|function| { - let arr = function.eval(&batch)?; + .map(|expr| { + let arr = expr.eval(&batch)?; Ok(arr.into_owned()) }) .collect::>>()?; let inputs = Batch::try_new(inputs)?; - state.function_state.poll_push(cx, inputs) + // Try to push first to avoid overwriting any buffered additional + // outputs. + // + // If we get a Pending, we need to return early with the original batch. + // + // TODO: Remove needing to do this, the clones should be cheap, but the + // expression execution is wasteful. + match state.function_state.poll_push(cx, inputs)? { + PollPush::Pending(_) => { + return Ok(PollPush::Pending(orig)); + } + other => { + // Batch was pushed to the function state, compute additional + // outputs. + let additional_outputs = self + .projected_outputs + .iter() + .map(|expr| { + let arr = expr.eval(&batch)?; + Ok(arr.into_owned()) + }) + .collect::>>()?; + + state.additional_outputs = additional_outputs; + + Ok(other) + } + } } fn poll_finalize_push( @@ -118,7 +155,35 @@ impl ExecutableOperator for PhysicalTableInOut { other => panic!("invalid partition state: {other:?}"), }; - state.function_state.poll_pull(cx) + match state.function_state.poll_pull(cx)? { + inout::InOutPollPull::Batch { batch, row_nums } => { + // We got a batch, append additional outputs according to + // returned row numbers. + if batch.num_rows() != row_nums.len() { + return Err(RayexecError::new("Row number mismatch").with_fields([ + ("batch_num_rows", batch.num_rows()), + ("row_nums_len", row_nums.len()), + ])); + } + + let selection = Arc::new(SelectionVector::from(row_nums)); + + let mut arrays = batch.into_arrays(); + arrays.reserve(state.additional_outputs.len()); + + for additional in &state.additional_outputs { + let mut additional = additional.clone(); + additional.select_mut(selection.clone()); + arrays.push(additional); + } + + let new_batch = Batch::try_new(arrays)?; + + Ok(PollPull::Computed(new_batch.into())) + } + inout::InOutPollPull::Pending => Ok(PollPull::Pending), + inout::InOutPollPull::Exhausted => Ok(PollPull::Exhausted), + } } } diff --git a/crates/rayexec_execution/src/functions/table/builtin/series.rs b/crates/rayexec_execution/src/functions/table/builtin/series.rs index e2c0c4196..556fe5536 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/series.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/series.rs @@ -13,7 +13,7 @@ use rayexec_error::{RayexecError, Result}; use crate::execution::operators::{PollFinalize, PollPull, PollPush}; use crate::expr::{self, Expression}; -use crate::functions::table::inout::{TableInOutFunction, TableInOutPartitionState}; +use crate::functions::table::inout::{InOutPollPull, TableInOutFunction, TableInOutPartitionState}; use crate::functions::table::{ InOutPlanner, PlannedTableFunction, @@ -122,6 +122,7 @@ impl TableInOutFunction for GenerateSeriesInOutImpl { finished: false, params: SeriesParams { exhausted: true, // Triggers param update on first pull + current_row_idx: 0, curr: 0, stop: 0, step: 0, @@ -140,6 +141,9 @@ impl TableInOutFunction for GenerateSeriesInOutImpl { struct SeriesParams { exhausted: bool, + /// Index of the row these parameters were generated from. + current_row_idx: usize, + curr: i64, stop: i64, step: i64, @@ -187,7 +191,7 @@ pub struct GenerateSeriesInOutPartitionState { batch_size: usize, /// Batch we're working on. batch: Option, - /// Row index we should try next. + /// Current row number next_row_idx: usize, /// If we're finished. finished: bool, @@ -223,13 +227,13 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { Ok(PollFinalize::Finalized) } - fn poll_pull(&mut self, cx: &mut Context) -> Result { + fn poll_pull(&mut self, cx: &mut Context) -> Result { if self.params.exhausted { let batch = match &self.batch { Some(batch) => batch, None => { if self.finished { - return Ok(PollPull::Exhausted); + return Ok(InOutPollPull::Exhausted); } // No batch to work on, come back later. @@ -237,7 +241,7 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { if let Some(push_waker) = self.push_waker.take() { push_waker.wake() } - return Ok(PollPull::Pending); + return Ok(InOutPollPull::Pending); } }; @@ -265,6 +269,7 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { self.params = SeriesParams { exhausted: false, + current_row_idx: self.next_row_idx, curr: start, stop: end, step, @@ -273,6 +278,7 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { _ => { self.params = SeriesParams { exhausted: false, + current_row_idx: self.next_row_idx, curr: 1, stop: 0, step: 1, @@ -291,6 +297,8 @@ impl TableInOutPartitionState for GenerateSeriesInOutPartitionState { let out = self.params.generate_next(self.batch_size); let batch = Batch::try_new([out])?; - Ok(PollPull::Computed(batch.into())) + let row_nums = vec![self.params.current_row_idx; batch.num_rows()]; + + Ok(InOutPollPull::Batch { batch, row_nums }) } } diff --git a/crates/rayexec_execution/src/functions/table/inout.rs b/crates/rayexec_execution/src/functions/table/inout.rs index 225a7686c..3ed5f7183 100644 --- a/crates/rayexec_execution/src/functions/table/inout.rs +++ b/crates/rayexec_execution/src/functions/table/inout.rs @@ -5,7 +5,7 @@ use dyn_clone::DynClone; use rayexec_bullet::batch::Batch; use rayexec_error::Result; -use crate::execution::operators::{PollFinalize, PollPull, PollPush}; +use crate::execution::operators::{PollFinalize, PollPush}; pub trait TableInOutFunction: Debug + Sync + Send + DynClone { fn create_states( @@ -14,10 +14,17 @@ pub trait TableInOutFunction: Debug + Sync + Send + DynClone { ) -> Result>>; } +#[derive(Debug)] +pub enum InOutPollPull { + Batch { batch: Batch, row_nums: Vec }, + Pending, + Exhausted, +} + pub trait TableInOutPartitionState: Debug + Sync + Send { fn poll_push(&mut self, cx: &mut Context, inputs: Batch) -> Result; fn poll_finalize_push(&mut self, cx: &mut Context) -> Result; - fn poll_pull(&mut self, cx: &mut Context) -> Result; + fn poll_pull(&mut self, cx: &mut Context) -> Result; } impl Clone for Box { diff --git a/crates/rayexec_execution/src/logical/logical_inout.rs b/crates/rayexec_execution/src/logical/logical_inout.rs index 88fb8edf0..248b5c1cb 100644 --- a/crates/rayexec_execution/src/logical/logical_inout.rs +++ b/crates/rayexec_execution/src/logical/logical_inout.rs @@ -11,9 +11,17 @@ use crate::functions::table::PlannedTableFunction; #[derive(Debug, Clone, PartialEq, Eq)] pub struct LogicalInOut { /// Table ref for referencing the output of this function. - pub table_ref: TableRef, + pub function_table_ref: TableRef, /// The table function. pub function: PlannedTableFunction, + /// Table ref for referencing the projected expressions. + /// + /// This only gets set during subquery decorrelation to project the original + /// inputs through the node. + pub projected_table_ref: Option, + /// Expressions that get projected out of this node alongside the results of + /// the table function. + pub projected_outputs: Vec, } impl Explainable for LogicalInOut { @@ -23,7 +31,13 @@ impl Explainable for LogicalInOut { .with_values_context("inputs", conf, &self.function.positional_inputs); if conf.verbose { - ent = ent.with_value("table_ref", self.table_ref); + ent = ent.with_value("function_table_ref", self.function_table_ref); + + if let Some(projected_table_ref) = self.projected_table_ref { + ent = ent + .with_value("projected_table_ref", projected_table_ref) + .with_values_context("projected_outputs", conf, &self.projected_outputs); + } } ent @@ -32,7 +46,11 @@ impl Explainable for LogicalInOut { impl LogicalNode for Node { fn get_output_table_refs(&self, _bind_context: &BindContext) -> Vec { - vec![self.node.table_ref] + if let Some(projected_table_ref) = self.node.projected_table_ref { + vec![self.node.function_table_ref, projected_table_ref] + } else { + vec![self.node.function_table_ref] + } } fn for_each_expr(&self, func: &mut F) -> Result<()> @@ -42,6 +60,10 @@ impl LogicalNode for Node { for expr in &self.node.function.positional_inputs { func(expr)? } + for expr in &self.node.projected_outputs { + func(expr)? + } + Ok(()) } @@ -52,6 +74,10 @@ impl LogicalNode for Node { for expr in &mut self.node.function.positional_inputs { func(expr)? } + for expr in &mut self.node.projected_outputs { + func(expr)? + } + Ok(()) } } diff --git a/crates/rayexec_execution/src/logical/planner/plan_from.rs b/crates/rayexec_execution/src/logical/planner/plan_from.rs index bb45e5221..a6a028f65 100644 --- a/crates/rayexec_execution/src/logical/planner/plan_from.rs +++ b/crates/rayexec_execution/src/logical/planner/plan_from.rs @@ -107,8 +107,10 @@ impl FromPlanner { // child as needed. Ok(LogicalOperator::InOut(Node { node: LogicalInOut { - table_ref: func.table_ref, + function_table_ref: func.table_ref, function: func.function, + projected_table_ref: None, + projected_outputs: Vec::new(), }, location: func.location, children: vec![LogicalOperator::EMPTY], diff --git a/crates/rayexec_execution/src/logical/planner/plan_subquery.rs b/crates/rayexec_execution/src/logical/planner/plan_subquery.rs index 00c942494..f73ce50be 100644 --- a/crates/rayexec_execution/src/logical/planner/plan_subquery.rs +++ b/crates/rayexec_execution/src/logical/planner/plan_subquery.rs @@ -57,7 +57,6 @@ impl SubqueryPlanner { mut conditions: Vec, lateral_columns: Vec, ) -> Result { - println!("PLANNING LATERAL"); // Very similar to planning correlated subqueries (becuase it is), just // we already have the correlated columns we're flattening for. @@ -806,8 +805,42 @@ impl DependentJoinPushdown { self.pushdown_children(bind_context, &mut inout.children)?; self.rewrite_expressions(&mut inout.node.function.positional_inputs)?; - // TODO: Need to check if we should propagate columns through - // the inout. If we do, it should just be a cross join. + // Add projections table as needed. + let table_ref = match inout.node.projected_table_ref { + Some(table_ref) => table_ref, // TODO: List out how this could be Some already + None => { + let table_ref = bind_context.new_ephemeral_table()?; + inout.node.projected_table_ref = Some(table_ref); + table_ref + } + }; + + // Append correlated columns to output projections. + let offset = inout.node.projected_outputs.len(); + for (idx, correlated) in self.columns.iter().enumerate() { + let expr = + Expression::Column(*self.column_map.get(correlated).ok_or_else(|| { + RayexecError::new( + format!("Missing correlated column in column map for appending projection to In/Out: {correlated:?}")) + })?); + + // Append column to table in bind context. + bind_context.push_column_for_table( + table_ref, + format!("__generated_inout_projection_decorrelation_{idx}"), + expr.datatype(bind_context.get_table_list())?, + )?; + + inout.node.projected_outputs.push(expr); + + self.column_map.insert( + correlated.clone(), + ColumnExpr { + table_scope: table_ref, + column: offset + idx, + }, + ); + } Ok(()) } diff --git a/slt/standard/functions/table/generate_series.slt b/slt/standard/functions/table/generate_series.slt index 85956682b..652a22471 100644 --- a/slt/standard/functions/table/generate_series.slt +++ b/slt/standard/functions/table/generate_series.slt @@ -100,7 +100,35 @@ select i * 2 from (select * from generate_series(1, 5)) as t(i); # Lateral input +query TT +describe select * from (values (3), (4)) v(a), generate_series(1, a) order by 1,2 +---- +a Int32 +generate_series Int64 + query II select * from (values (3), (4)) v(a), generate_series(1, a) order by 1,2; ---- +3 1 +3 2 +3 3 +4 1 +4 2 +4 3 +4 4 + +# TODO: Not yet implemented: dependent join pushdown for node: CrossJoin ... +# Also need to be careful since this is a nested lateral. +# query III +# select * from (values (3), (4)) v(a), generate_series(1, a) g1, generate_series(a, 5) g2 order by 1,2,3; +# ---- +query III +select * from (values (3, 6), (4, 5)) v(a, b), generate_series(a, b) order by 1,2,3; +---- +3 6 3 +3 6 4 +3 6 5 +3 6 6 +4 5 4 +4 5 5 From e631142e02b2c683e465e9e99770e5b8378dfd4e Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 15:43:04 -0600 Subject: [PATCH 15/23] read_csv --- crates/rayexec_csv/src/read_csv.rs | 157 ++++++------------ .../src/execution/operators/table_function.rs | 5 +- .../src/functions/table/builtin/refresh.rs | 38 +---- .../src/functions/table/builtin/system.rs | 108 ++++++------ .../src/functions/table/mod.rs | 71 +++++++- .../src/functions/table/scan.rs | 67 -------- 6 files changed, 175 insertions(+), 271 deletions(-) delete mode 100644 crates/rayexec_execution/src/functions/table/scan.rs diff --git a/crates/rayexec_csv/src/read_csv.rs b/crates/rayexec_csv/src/read_csv.rs index 5ceef770b..4b8d31627 100644 --- a/crates/rayexec_csv/src/read_csv.rs +++ b/crates/rayexec_csv/src/read_csv.rs @@ -1,19 +1,25 @@ +use std::collections::HashMap; +use std::sync::Arc; + use futures::future::BoxFuture; -use futures::StreamExt; +use futures::{FutureExt, StreamExt}; use rayexec_bullet::datatype::DataTypeId; -use rayexec_bullet::field::Schema; +use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_error::{RayexecError, Result}; use rayexec_execution::database::DatabaseContext; -use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; +use rayexec_execution::expr; +use rayexec_execution::functions::table::{ + try_location_and_access_config_from_args, + PlannedTableFunction, + ScanPlanner, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, +}; use rayexec_execution::functions::{FunctionInfo, Signature}; +use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; -use rayexec_execution::storage::table_storage::DataTable; -use rayexec_io::location::{AccessConfig, FileLocation}; use rayexec_io::{FileProvider, FileSource}; -use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; -use rayexec_proto::ProtoConv; -use serde::{Deserialize, Serialize}; use crate::datatable::SingleFileCsvDataTable; use crate::decoder::{CsvDecoder, DecoderState}; @@ -43,79 +49,33 @@ impl FunctionInfo for ReadCsv { } impl TableFunction for ReadCsv { - fn plan_and_initialize<'a>( - &self, - _context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - Box::pin(ReadCsvImpl::initialize(self.clone(), args)) - } - - fn decode_state(&self, state: &[u8]) -> Result> { - let state = ReadCsvState::decode(state)?; - Ok(Box::new(ReadCsvImpl { - func: self.clone(), - state, - })) + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::Scan(self) } } -#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] -struct ReadCsvState { - location: FileLocation, - conf: AccessConfig, - csv_schema: CsvSchema, - dialect: DialectOptions, -} - -impl ReadCsvState { - fn encode(&self, buf: &mut Vec) -> Result<()> { - let mut packed = PackedEncoder::new(buf); - packed.encode_next(&self.location.to_proto()?)?; - packed.encode_next(&self.conf.to_proto()?)?; - packed.encode_next(&self.csv_schema.schema.to_proto()?)?; - packed.encode_next(&self.csv_schema.has_header)?; - packed.encode_next(&self.csv_schema.has_header)?; - packed.encode_next(&(self.dialect.delimiter as i32))?; - packed.encode_next(&(self.dialect.quote as i32))?; - Ok(()) - } - - fn decode(buf: &[u8]) -> Result { - let mut packed = PackedDecoder::new(buf); - let location = FileLocation::from_proto(packed.decode_next()?)?; - let conf = AccessConfig::from_proto(packed.decode_next()?)?; - let schema = Schema::from_proto(packed.decode_next()?)?; - let has_header: bool = packed.decode_next()?; - let delimiter: i32 = packed.decode_next()?; - let quote: i32 = packed.decode_next()?; - - Ok(ReadCsvState { - location, - conf, - csv_schema: CsvSchema { schema, has_header }, - dialect: DialectOptions { - delimiter: delimiter as u8, - quote: quote as u8, - }, - }) +impl ScanPlanner for ReadCsv { + fn plan<'a>( + &self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result> { + Self::plan_inner(self.clone(), context, positional_inputs, named_inputs).boxed() } } -#[derive(Debug, Clone, PartialEq, Eq)] -struct ReadCsvImpl { - func: ReadCsv, - state: ReadCsvState, -} - -impl ReadCsvImpl { - async fn initialize( - func: ReadCsv, - args: TableFunctionInputs, - ) -> Result> { - let (location, conf) = args.try_location_and_access_config()?; +impl ReadCsv { + async fn plan_inner<'a>( + self: Self, + _context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { + let (location, conf) = + try_location_and_access_config_from_args(&self, &positional_inputs, &named_inputs)?; - let mut source = func + let mut source = self .runtime .file_provider() .file_source(location.clone(), &conf)?; @@ -143,38 +103,23 @@ impl ReadCsvImpl { let completed = state.completed_records(); let csv_schema = CsvSchema::infer_from_records(completed)?; - Ok(Box::new(Self { - func, - state: ReadCsvState { - location, - conf, - dialect, - csv_schema, - }, - })) - } -} - -impl PlannedTableFunction2 for ReadCsvImpl { - fn table_function(&self) -> &dyn TableFunction { - &self.func - } - - fn encode_state(&self, state: &mut Vec) -> Result<()> { - self.state.encode(state) - } + let schema = csv_schema.schema.clone(); - fn schema(&self) -> Schema { - self.state.csv_schema.schema.clone() - } + let datatable = SingleFileCsvDataTable { + options: dialect, + csv_schema, + location, + conf, + runtime: self.runtime.clone(), + }; - fn datatable(&self) -> Result> { - Ok(Box::new(SingleFileCsvDataTable { - options: self.state.dialect, - csv_schema: self.state.csv_schema.clone(), - location: self.state.location.clone(), - conf: self.state.conf.clone(), - runtime: self.func.runtime.clone(), - })) + Ok(PlannedTableFunction { + function: Box::new(self), + positional_inputs: positional_inputs.into_iter().map(expr::lit).collect(), + named_inputs, + function_impl: TableFunctionImpl::Scan(Arc::new(datatable)), + cardinality: StatisticsValue::Unknown, + schema, + }) } } diff --git a/crates/rayexec_execution/src/execution/operators/table_function.rs b/crates/rayexec_execution/src/execution/operators/table_function.rs index fcd4c9444..c3e4de288 100644 --- a/crates/rayexec_execution/src/execution/operators/table_function.rs +++ b/crates/rayexec_execution/src/execution/operators/table_function.rs @@ -20,13 +20,12 @@ use super::{ }; use crate::database::DatabaseContext; use crate::explain::explainable::{ExplainConfig, ExplainEntry, Explainable}; -use crate::functions::table::scan::TableScanState; use crate::functions::table::{PlannedTableFunction, TableFunctionImpl}; use crate::proto::DatabaseProtoConv; -use crate::storage::table_storage::Projections; +use crate::storage::table_storage::{DataTableScan, Projections}; pub struct TableFunctionPartitionState { - scan_state: Box, + scan_state: Box, /// In progress pull we're working on. future: Option>>>, } diff --git a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs b/crates/rayexec_execution/src/functions/table/builtin/refresh.rs index 7521bf3f2..fa750739d 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/refresh.rs @@ -9,10 +9,9 @@ use rayexec_error::{RayexecError, Result}; use crate::database::memory_catalog::MemoryCatalog; use crate::database::DatabaseContext; -use crate::functions::table::{PlannedTableFunction2, TableFunction, TableFunctionInputs}; +use crate::functions::table::{TableFunction, TableFunctionInputs, TableFunctionPlanner}; use crate::functions::{FunctionInfo, Signature}; use crate::storage::catalog_storage::CatalogStorage; -use crate::storage::table_storage::DataTable; pub trait RefreshOperation: Debug + Clone + Copy + PartialEq + Eq + Sync + Send + 'static { const NAME: &'static str; @@ -96,40 +95,7 @@ impl FunctionInfo for RefreshObjects { } impl TableFunction for RefreshObjects { - fn plan_and_initialize<'a>( - &self, - _context: &'a DatabaseContext, - _args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - unimplemented!() - } - - fn decode_state(&self, _state: &[u8]) -> Result> { - unimplemented!() - } -} - -#[derive(Debug, Clone)] -pub struct RefreshObjectsImpl { - func: RefreshObjects, - _state: Option, - _op: PhantomData, -} - -impl PlannedTableFunction2 for RefreshObjectsImpl { - fn encode_state(&self, _state: &mut Vec) -> Result<()> { - Ok(()) - } - - fn table_function(&self) -> &dyn TableFunction { - &self.func - } - - fn schema(&self) -> Schema { - O::schema() - } - - fn datatable(&self) -> Result> { + fn planner(&self) -> TableFunctionPlanner { unimplemented!() } } diff --git a/crates/rayexec_execution/src/functions/table/builtin/system.rs b/crates/rayexec_execution/src/functions/table/builtin/system.rs index 7d869d332..b585f67f2 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/system.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/system.rs @@ -1,4 +1,4 @@ -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::fmt::Debug; use std::marker::PhantomData; use std::sync::Arc; @@ -7,9 +7,10 @@ use futures::future::BoxFuture; use parking_lot::Mutex; use rayexec_bullet::array::Array; use rayexec_bullet::batch::Batch; -use rayexec_bullet::datatype::DataType; +use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_bullet::executor::builder::{ArrayDataBuffer, GermanVarlenBuffer}; use rayexec_bullet::field::{Field, Schema}; +use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_bullet::storage::GermanVarlenStorage; use rayexec_error::{OptionExt, RayexecError, Result}; @@ -17,8 +18,16 @@ use crate::database::catalog::CatalogTx; use crate::database::catalog_entry::{CatalogEntryInner, CatalogEntryType}; use crate::database::memory_catalog::MemoryCatalog; use crate::database::{AttachInfo, DatabaseContext}; -use crate::functions::table::{PlannedTableFunction2, TableFunction, TableFunctionInputs}; +use crate::expr; +use crate::functions::table::{ + PlannedTableFunction, + ScanPlanner, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, +}; use crate::functions::{FunctionInfo, Signature}; +use crate::logical::statistics::StatisticsValue; use crate::storage::table_storage::{ DataTable, DataTableScan, @@ -248,18 +257,35 @@ impl FunctionInfo for SystemFunction { } fn signatures(&self) -> &[Signature] { - unimplemented!() + &[Signature { + positional_args: &[], + variadic_arg: None, + return_type: DataTypeId::Any, + }] } } impl TableFunction for SystemFunction { - fn plan_and_initialize<'a>( + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::Scan(&SystemFunctionPlanner:: { _f: PhantomData }) + } +} + +#[derive(Debug, Clone)] +pub struct SystemFunctionPlanner { + _f: PhantomData, +} + +impl ScanPlanner for SystemFunctionPlanner +where + F: SystemFunctionImpl, +{ + fn plan<'a>( &self, context: &'a DatabaseContext, - _args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - // TODO: Method on args returning an error if not empty. - + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result> { let databases = context .iter_databases() .map(|(name, database)| { @@ -271,55 +297,27 @@ impl TableFunction for SystemFunction { }) .collect(); - let function = *self; - Box::pin(async move { - Ok(Box::new(PlannedSystemFunction { - databases, - function, - }) as _) - }) - } - - fn decode_state(&self, _state: &[u8]) -> Result> { - Ok(Box::new(PlannedSystemFunction { - databases: Vec::new(), - function: *self, - })) + let planned = PlannedTableFunction { + function: Box::new(SystemFunction::::new()), + positional_inputs: positional_inputs.into_iter().map(expr::lit).collect(), + named_inputs, + function_impl: TableFunctionImpl::Scan(Arc::new(SystemDataTable:: { + databases: Arc::new(Mutex::new(Some(databases))), + _f: PhantomData, + })), + cardinality: StatisticsValue::Unknown, + schema: F::schema(), + }; + + Box::pin(async move { Ok(planned) }) } } #[derive(Debug, Clone)] -pub struct PlannedSystemFunction { - databases: Vec<(String, Arc, Option)>, - function: SystemFunction, -} - -impl PlannedTableFunction2 for PlannedSystemFunction { - fn encode_state(&self, _state: &mut Vec) -> Result<()> { - Ok(()) - } - - fn table_function(&self) -> &dyn TableFunction { - &self.function - } - - fn schema(&self) -> Schema { - F::schema() - } - - fn datatable(&self) -> Result> { - Ok(Box::new(SystemDataTable { - databases: Mutex::new(Some(self.databases.clone().into_iter().collect())), - function: self.function, - })) - } -} - -#[derive(Debug)] struct SystemDataTable { #[allow(clippy::type_complexity)] // Temp - databases: Mutex, Option)>>>, - function: SystemFunction, + databases: Arc, Option)>>>>, + _f: PhantomData, } impl DataTable for SystemDataTable { @@ -335,9 +333,9 @@ impl DataTable for SystemDataTable { .ok_or_else(|| RayexecError::new("Scan called multiple times"))?; let mut scans: Vec> = vec![Box::new(ProjectedScan::new( - SystemDataTableScan { + SystemDataTableScan:: { databases, - _function: self.function, + _f: PhantomData, }, projections, )) as _]; @@ -351,7 +349,7 @@ impl DataTable for SystemDataTable { #[derive(Debug)] struct SystemDataTableScan { databases: VecDeque<(String, Arc, Option)>, - _function: SystemFunction, + _f: PhantomData, } impl DataTableScan for SystemDataTableScan { diff --git a/crates/rayexec_execution/src/functions/table/mod.rs b/crates/rayexec_execution/src/functions/table/mod.rs index e7cd55321..d6823890b 100644 --- a/crates/rayexec_execution/src/functions/table/mod.rs +++ b/crates/rayexec_execution/src/functions/table/mod.rs @@ -1,10 +1,10 @@ pub mod builtin; pub mod inout; pub mod inputs; -pub mod scan; use std::collections::HashMap; use std::fmt::Debug; +use std::sync::Arc; use dyn_clone::DynClone; use futures::future::BoxFuture; @@ -13,8 +13,10 @@ use inout::TableInOutFunction; use inputs::TableFunctionInputs; use rayexec_bullet::field::Schema; use rayexec_bullet::scalar::OwnedScalarValue; -use rayexec_error::Result; -use scan::TableScanFunction; +use rayexec_error::{RayexecError, Result}; +use rayexec_io::location::{AccessConfig, FileLocation}; +use rayexec_io::s3::credentials::AwsCredentials; +use rayexec_io::s3::S3Location; use super::FunctionInfo; use crate::database::DatabaseContext; @@ -146,7 +148,7 @@ impl Eq for PlannedTableFunction {} #[derive(Debug, Clone)] pub enum TableFunctionImpl { /// Table function that produces a table as its output. - Scan(Box), + Scan(Arc), /// A table function that accepts dynamic arguments and produces a table /// output. InOut(Box), @@ -205,3 +207,64 @@ impl Clone for Box { dyn_clone::clone_box(&**self) } } + +/// Try to get a file location and access config from the table args. +// TODO: Secrets provider that we pass in allowing us to get creds from some +// secrets store. +pub fn try_location_and_access_config_from_args( + func: &impl TableFunction, + positional: &[OwnedScalarValue], + named: &HashMap, +) -> Result<(FileLocation, AccessConfig)> { + let loc = match positional.first() { + Some(loc) => { + let loc = loc.try_as_str()?; + FileLocation::parse(loc) + } + None => { + return Err(RayexecError::new(format!( + "Expected at least one position argument for function {}", + func.name(), + ))) + } + }; + + let conf = match &loc { + FileLocation::Url(url) => { + if S3Location::is_s3_location(url) { + let key_id = try_get_named(func, "key_id", named)? + .try_as_str()? + .to_string(); + let secret = try_get_named(func, "secret", named)? + .try_as_str()? + .to_string(); + let region = try_get_named(func, "region", named)? + .try_as_str()? + .to_string(); + + AccessConfig::S3 { + credentials: AwsCredentials { key_id, secret }, + region, + } + } else { + AccessConfig::None + } + } + FileLocation::Path(_) => AccessConfig::None, + }; + + Ok((loc, conf)) +} + +pub fn try_get_named<'a>( + func: &impl TableFunction, + name: &str, + named: &'a HashMap, +) -> Result<&'a OwnedScalarValue> { + named.get(name).ok_or_else(|| { + RayexecError::new(format!( + "Expected named argument '{name}' for function {}", + func.name() + )) + }) +} diff --git a/crates/rayexec_execution/src/functions/table/scan.rs b/crates/rayexec_execution/src/functions/table/scan.rs deleted file mode 100644 index 122cf5bf5..000000000 --- a/crates/rayexec_execution/src/functions/table/scan.rs +++ /dev/null @@ -1,67 +0,0 @@ -use std::fmt::Debug; - -use dyn_clone::DynClone; -use futures::future::BoxFuture; -use rayexec_bullet::batch::Batch; -use rayexec_error::Result; - -use crate::storage::table_storage::Projections; - -pub trait TableScanFunction: Debug + Sync + Send + DynClone { - fn scan( - &self, - projections: Projections, - num_partitions: usize, - ) -> Result>>; -} - -pub trait TableScanState: Debug + Sync + Send { - fn pull(&mut self) -> BoxFuture<'_, Result>>; -} - -impl Clone for Box { - fn clone(&self) -> Self { - dyn_clone::clone_box(&**self) - } -} - -/// Helper for wrapping an unprojected scan with a projections list to produce -/// projected batches. -/// -/// This is inefficient compared to handling the projection in the scan itself -/// since this projects a batch after it's already been read. -#[derive(Debug)] -pub struct ProjectedTableScanState { - pub projections: Projections, - pub scan_state: S, -} - -impl ProjectedTableScanState { - pub fn new(scan_state: S, projections: Projections) -> Self { - ProjectedTableScanState { - projections, - scan_state, - } - } - - async fn pull_inner(&mut self) -> Result> { - let batch = match self.scan_state.pull().await? { - Some(batch) => batch, - None => return Ok(None), - }; - - match self.projections.column_indices.as_ref() { - Some(indices) => { - let batch = batch.project(indices); - Ok(Some(batch)) - } - None => Ok(Some(batch)), - } - } -} - -impl TableScanState for ProjectedTableScanState { - fn pull(&mut self) -> BoxFuture<'_, Result>> { - Box::pin(async { self.pull_inner().await }) - } -} From 1402b0b94fd95417b240364c2effe628c91ca4de Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 15:52:19 -0600 Subject: [PATCH 16/23] read_postgres --- crates/rayexec_postgres/src/read_postgres.rs | 154 ++++++------------- 1 file changed, 51 insertions(+), 103 deletions(-) diff --git a/crates/rayexec_postgres/src/read_postgres.rs b/crates/rayexec_postgres/src/read_postgres.rs index a3468ccd1..b3adf6a25 100644 --- a/crates/rayexec_postgres/src/read_postgres.rs +++ b/crates/rayexec_postgres/src/read_postgres.rs @@ -1,16 +1,24 @@ +use std::collections::HashMap; +use std::sync::Arc; + use futures::future::BoxFuture; +use futures::FutureExt; use rayexec_bullet::datatype::DataTypeId; use rayexec_bullet::field::Schema; -use rayexec_error::{OptionExt, RayexecError, Result}; +use rayexec_bullet::scalar::OwnedScalarValue; +use rayexec_error::{RayexecError, Result}; use rayexec_execution::database::DatabaseContext; -use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; +use rayexec_execution::expr; +use rayexec_execution::functions::table::{ + PlannedTableFunction, + ScanPlanner, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, +}; use rayexec_execution::functions::{FunctionInfo, Signature}; +use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; -use rayexec_execution::storage::table_storage::DataTable; -use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; -use rayexec_proto::ProtoConv; -use serde::{Deserialize, Serialize}; use crate::{PostgresClient, PostgresDataTable}; @@ -34,82 +42,43 @@ impl FunctionInfo for ReadPostgres { } impl TableFunction for ReadPostgres { - fn plan_and_initialize<'a>( - &self, - _context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - Box::pin(ReadPostgresImpl::initialize(self.clone(), args)) + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::Scan(self) } - - fn decode_state(&self, state: &[u8]) -> Result> { - Ok(Box::new(ReadPostgresImpl { - func: self.clone(), - state: ReadPostgresState::decode(state)?, - client: None, - })) - } -} - -#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] -struct ReadPostgresState { - conn_str: String, - schema: String, - table: String, - table_schema: Schema, } -impl ReadPostgresState { - fn encode(&self, buf: &mut Vec) -> Result<()> { - let mut packed = PackedEncoder::new(buf); - packed.encode_next(&self.conn_str)?; - packed.encode_next(&self.schema)?; - packed.encode_next(&self.table)?; - packed.encode_next(&self.table_schema.to_proto()?)?; - Ok(()) - } - - fn decode(buf: &[u8]) -> Result { - let mut packed = PackedDecoder::new(buf); - Ok(ReadPostgresState { - conn_str: packed.decode_next()?, - schema: packed.decode_next()?, - table: packed.decode_next()?, - table_schema: Schema::from_proto(packed.decode_next()?)?, - }) +impl ScanPlanner for ReadPostgres { + fn plan<'a>( + &self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result> { + Self::plan_inner(self.clone(), context, positional_inputs, named_inputs).boxed() } } -#[derive(Debug, Clone)] -struct ReadPostgresImpl { - func: ReadPostgres, - state: ReadPostgresState, - client: Option, -} - -impl ReadPostgresImpl -where - R: Runtime, -{ - async fn initialize( - func: ReadPostgres, - args: TableFunctionInputs, - ) -> Result> { - if !args.named.is_empty() { +impl ReadPostgres { + async fn plan_inner<'a>( + self: Self, + _context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { + if !named_inputs.is_empty() { return Err(RayexecError::new( "read_postgres does not accept named arguments", )); } - if args.positional.len() != 3 { + if positional_inputs.len() != 3 { return Err(RayexecError::new("read_postgres requires 3 arguments")); } - let mut args = args.clone(); - let table = args.positional.pop().unwrap().try_into_string()?; - let schema = args.positional.pop().unwrap().try_into_string()?; - let conn_str = args.positional.pop().unwrap().try_into_string()?; + let conn_str = positional_inputs.get(0).unwrap().try_as_str()?; + let schema = positional_inputs.get(1).unwrap().try_as_str()?; + let table = positional_inputs.get(2).unwrap().try_as_str()?; - let client = PostgresClient::connect(&conn_str, &func.runtime).await?; + let client = PostgresClient::connect(conn_str, &self.runtime).await?; let fields = match client.get_fields_and_types(&schema, &table).await? { Some((fields, _)) => fields, @@ -118,40 +87,19 @@ where let table_schema = Schema::new(fields); - Ok(Box::new(ReadPostgresImpl { - func, - state: ReadPostgresState { - conn_str, - schema, - table, - table_schema, - }, - client: Some(client), - })) - } -} - -impl PlannedTableFunction2 for ReadPostgresImpl -where - R: Runtime, -{ - fn table_function(&self) -> &dyn TableFunction { - &self.func - } - - fn schema(&self) -> Schema { - self.state.table_schema.clone() - } - - fn encode_state(&self, state: &mut Vec) -> Result<()> { - self.state.encode(state) - } + let datatable = PostgresDataTable { + client, + schema: schema.to_string(), + table: table.to_string(), + }; - fn datatable(&self) -> Result> { - Ok(Box::new(PostgresDataTable { - client: self.client.as_ref().required("postgres client")?.clone(), - schema: self.state.schema.clone(), - table: self.state.table.clone(), - })) + Ok(PlannedTableFunction { + function: Box::new(self), + positional_inputs: positional_inputs.into_iter().map(expr::lit).collect(), + named_inputs, + function_impl: TableFunctionImpl::Scan(Arc::new(datatable)), + cardinality: StatisticsValue::Unknown, + schema: table_schema, + }) } } From 7a8a5aa7e33744d855812cd1d5bf5605775d5280 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 15:58:46 -0600 Subject: [PATCH 17/23] read_parquet --- .../src/functions/read_parquet.rs | 155 ++++++------------ 1 file changed, 49 insertions(+), 106 deletions(-) diff --git a/crates/rayexec_parquet/src/functions/read_parquet.rs b/crates/rayexec_parquet/src/functions/read_parquet.rs index e0a019d09..55a598f3a 100644 --- a/crates/rayexec_parquet/src/functions/read_parquet.rs +++ b/crates/rayexec_parquet/src/functions/read_parquet.rs @@ -1,21 +1,25 @@ +use std::collections::HashMap; use std::sync::Arc; -use bytes::Bytes; use futures::future::BoxFuture; +use futures::FutureExt; use rayexec_bullet::datatype::DataTypeId; -use rayexec_bullet::field::Schema; +use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_error::Result; use rayexec_execution::database::DatabaseContext; -use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; +use rayexec_execution::expr; +use rayexec_execution::functions::table::{ + try_location_and_access_config_from_args, + PlannedTableFunction, + ScanPlanner, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, +}; use rayexec_execution::functions::{FunctionInfo, Signature}; use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; -use rayexec_execution::storage::table_storage::DataTable; -use rayexec_io::location::{AccessConfig, FileLocation}; use rayexec_io::FileProvider; -use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; -use rayexec_proto::ProtoConv; use super::datatable::RowGroupPartitionedDataTable; use crate::metadata::Metadata; @@ -45,71 +49,33 @@ impl FunctionInfo for ReadParquet { } impl TableFunction for ReadParquet { - fn plan_and_initialize<'a>( - &self, - _context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - Box::pin(ReadParquetImpl::initialize(self.clone(), args)) - } - - fn decode_state(&self, state: &[u8]) -> Result> { - Ok(Box::new(ReadParquetImpl { - func: self.clone(), - state: ReadParquetState::decode(state)?, - })) + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::Scan(self) } } -#[derive(Debug, Clone)] -struct ReadParquetState { - location: FileLocation, - conf: AccessConfig, - metadata: Arc, - schema: Schema, -} - -impl ReadParquetState { - fn encode(&self, buf: &mut Vec) -> Result<()> { - let mut packed = PackedEncoder::new(buf); - packed.encode_next(&self.location.to_proto()?)?; - packed.encode_next(&self.conf.to_proto()?)?; - packed.encode_next(&self.metadata.metadata_buffer)?; - packed.encode_next(&self.schema.to_proto()?)?; - Ok(()) - } - - fn decode(buf: &[u8]) -> Result { - let mut packed = PackedDecoder::new(buf); - let location = FileLocation::from_proto(packed.decode_next()?)?; - let conf = AccessConfig::from_proto(packed.decode_next()?)?; - let metadata_buffer: Bytes = packed.decode_next()?; - let schema = Schema::from_proto(packed.decode_next()?)?; - - let metadata = Arc::new(Metadata::try_from_buffer(metadata_buffer)?); - - Ok(ReadParquetState { - location, - conf, - schema, - metadata, - }) +impl ScanPlanner for ReadParquet { + fn plan<'a>( + &self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result> { + Self::plan_inner(self.clone(), context, positional_inputs, named_inputs).boxed() } } -#[derive(Debug, Clone)] -pub struct ReadParquetImpl { - func: ReadParquet, - state: ReadParquetState, -} +impl ReadParquet { + async fn plan_inner<'a>( + self: Self, + _context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { + let (location, conf) = + try_location_and_access_config_from_args(&self, &positional_inputs, &named_inputs)?; -impl ReadParquetImpl { - async fn initialize( - func: ReadParquet, - args: TableFunctionInputs, - ) -> Result> { - let (location, conf) = args.try_location_and_access_config()?; - let mut source = func + let mut source = self .runtime .file_provider() .file_source(location.clone(), &conf)?; @@ -119,51 +85,28 @@ impl ReadParquetImpl { let metadata = Metadata::new_from_source(source.as_mut(), size).await?; let schema = from_parquet_schema(metadata.decoded_metadata.file_metadata().schema_descr())?; - Ok(Box::new(Self { - func, - state: ReadParquetState { - location, - conf, - metadata: Arc::new(metadata), - schema, - }, - })) - } -} - -impl PlannedTableFunction2 for ReadParquetImpl { - fn table_function(&self) -> &dyn TableFunction { - &self.func - } - - fn schema(&self) -> Schema { - self.state.schema.clone() - } - - fn encode_state(&self, state: &mut Vec) -> Result<()> { - self.state.encode(state) - } - - fn cardinality(&self) -> StatisticsValue { - let num_rows = self - .state - .metadata + let num_rows = metadata .decoded_metadata .row_groups() .iter() .map(|g| g.num_rows()) .sum::() as usize; - StatisticsValue::Exact(num_rows) - } - - fn datatable(&self) -> Result> { - Ok(Box::new(RowGroupPartitionedDataTable { - metadata: self.state.metadata.clone(), - schema: self.state.schema.clone(), - location: self.state.location.clone(), - conf: self.state.conf.clone(), - runtime: self.func.runtime.clone(), - })) + let datatable = RowGroupPartitionedDataTable { + metadata: Arc::new(metadata), + schema: schema.clone(), + location, + conf, + runtime: self.runtime.clone(), + }; + + Ok(PlannedTableFunction { + function: Box::new(self), + positional_inputs: positional_inputs.into_iter().map(expr::lit).collect(), + named_inputs, + function_impl: TableFunctionImpl::Scan(Arc::new(datatable)), + cardinality: StatisticsValue::Exact(num_rows), + schema, + }) } } From b5f9d73220f18baf9bb9e42ac7bda8179572c5cd Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 16:03:55 -0600 Subject: [PATCH 18/23] read_delta --- crates/rayexec_delta/src/read_delta.rs | 145 ++++++++----------------- 1 file changed, 44 insertions(+), 101 deletions(-) diff --git a/crates/rayexec_delta/src/read_delta.rs b/crates/rayexec_delta/src/read_delta.rs index 55d15cb80..fcd7d5d1c 100644 --- a/crates/rayexec_delta/src/read_delta.rs +++ b/crates/rayexec_delta/src/read_delta.rs @@ -1,18 +1,24 @@ +use std::collections::HashMap; use std::sync::Arc; use futures::future::BoxFuture; +use futures::FutureExt; use rayexec_bullet::datatype::DataTypeId; -use rayexec_bullet::field::Schema; -use rayexec_error::{RayexecError, Result}; +use rayexec_bullet::scalar::OwnedScalarValue; +use rayexec_error::Result; use rayexec_execution::database::DatabaseContext; -use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; +use rayexec_execution::expr; +use rayexec_execution::functions::table::{ + try_location_and_access_config_from_args, + PlannedTableFunction, + ScanPlanner, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, +}; use rayexec_execution::functions::{FunctionInfo, Signature}; +use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; -use rayexec_execution::storage::table_storage::DataTable; -use rayexec_io::location::{AccessConfig, FileLocation}; -use rayexec_proto::packed::{PackedDecoder, PackedEncoder}; -use rayexec_proto::ProtoConv; use crate::datatable::DeltaDataTable; use crate::protocol::table::Table; @@ -41,109 +47,46 @@ impl FunctionInfo for ReadDelta { } impl TableFunction for ReadDelta { - fn plan_and_initialize<'a>( - &self, - _context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - let func = self.clone(); - Box::pin(async move { ReadDeltaImpl::initialize(func, args).await }) - } - - fn decode_state(&self, state: &[u8]) -> Result> { - Ok(Box::new(ReadDeltaImpl { - func: self.clone(), - state: ReadDeltaState::decode(state)?, - })) + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::Scan(self) } } -#[derive(Debug, Clone)] -struct ReadDeltaState { - location: FileLocation, - conf: AccessConfig, - schema: Schema, - table: Option>, // Populate on re-init if needed. -} - -impl ReadDeltaState { - fn encode(&self, buf: &mut Vec) -> Result<()> { - let mut packed = PackedEncoder::new(buf); - packed.encode_next(&self.location.to_proto()?)?; - packed.encode_next(&self.conf.to_proto()?)?; - packed.encode_next(&self.schema.to_proto()?)?; - Ok(()) - } - - fn decode(buf: &[u8]) -> Result { - let mut packed = PackedDecoder::new(buf); - let location = FileLocation::from_proto(packed.decode_next()?)?; - let conf = AccessConfig::from_proto(packed.decode_next()?)?; - let schema = Schema::from_proto(packed.decode_next()?)?; - Ok(ReadDeltaState { - location, - conf, - schema, - table: None, - }) +impl ScanPlanner for ReadDelta { + fn plan<'a>( + &self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result> { + Self::plan_inner(self.clone(), context, positional_inputs, named_inputs).boxed() } } -#[derive(Debug, Clone)] -pub struct ReadDeltaImpl { - func: ReadDelta, - state: ReadDeltaState, -} - -impl ReadDeltaImpl { - async fn initialize( - func: ReadDelta, - args: TableFunctionInputs, - ) -> Result> { - let (location, conf) = args.try_location_and_access_config()?; +impl ReadDelta { + async fn plan_inner<'a>( + self: Self, + _context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { + let (location, conf) = + try_location_and_access_config_from_args(&self, &positional_inputs, &named_inputs)?; - let provider = func.runtime.file_provider(); + let provider = self.runtime.file_provider(); let table = Table::load(location.clone(), provider, conf.clone()).await?; let schema = table.table_schema()?; - Ok(Box::new(ReadDeltaImpl { - func, - state: ReadDeltaState { - location, - conf, - schema, - table: Some(Arc::new(table)), - }, - })) - } -} - -impl PlannedTableFunction2 for ReadDeltaImpl { - fn reinitialize(&self) -> BoxFuture> { - // TODO: Reinit table. - // TODO: Needs mut - unimplemented!() - } - - fn table_function(&self) -> &dyn TableFunction { - &self.func - } - - fn encode_state(&self, state: &mut Vec) -> Result<()> { - self.state.encode(state) - } - - fn schema(&self) -> Schema { - self.state.schema.clone() - } - - fn datatable(&self) -> Result> { - let table = match self.state.table.as_ref() { - Some(table) => table.clone(), - None => return Err(RayexecError::new("Delta table not initialized")), - }; - - Ok(Box::new(DeltaDataTable { table })) + Ok(PlannedTableFunction { + function: Box::new(self), + positional_inputs: positional_inputs.into_iter().map(expr::lit).collect(), + named_inputs, + function_impl: TableFunctionImpl::Scan(Arc::new(DeltaDataTable { + table: Arc::new(table), // TODO: Arc Arc + })), + cardinality: StatisticsValue::Unknown, + schema, + }) } } From 81a714ab83ff47052e4bd37349c90939981b0934 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 16:22:10 -0600 Subject: [PATCH 19/23] unity stuff --- .../src/functions/table/mod.rs | 13 ++ crates/rayexec_iceberg/src/read_iceberg.rs | 119 +++++++---------- crates/rayexec_unity_catalog/src/functions.rs | 122 ++++++++++-------- 3 files changed, 126 insertions(+), 128 deletions(-) diff --git a/crates/rayexec_execution/src/functions/table/mod.rs b/crates/rayexec_execution/src/functions/table/mod.rs index d6823890b..b5b15e49b 100644 --- a/crates/rayexec_execution/src/functions/table/mod.rs +++ b/crates/rayexec_execution/src/functions/table/mod.rs @@ -268,3 +268,16 @@ pub fn try_get_named<'a>( )) }) } + +pub fn try_get_positional<'a>( + func: &impl TableFunction, + pos: usize, + positional: &'a [OwnedScalarValue], +) -> Result<&'a OwnedScalarValue> { + positional.get(pos).ok_or_else(|| { + RayexecError::new(format!( + "Expected argument at position {pos} for function {}", + func.name() + )) + }) +} diff --git a/crates/rayexec_iceberg/src/read_iceberg.rs b/crates/rayexec_iceberg/src/read_iceberg.rs index 44d53e95e..f7283c0ff 100644 --- a/crates/rayexec_iceberg/src/read_iceberg.rs +++ b/crates/rayexec_iceberg/src/read_iceberg.rs @@ -1,16 +1,24 @@ +use std::collections::HashMap; use std::sync::Arc; use futures::future::BoxFuture; +use futures::FutureExt; use rayexec_bullet::datatype::DataTypeId; -use rayexec_bullet::field::Schema; -use rayexec_error::{not_implemented, RayexecError, Result}; +use rayexec_bullet::scalar::OwnedScalarValue; +use rayexec_error::Result; use rayexec_execution::database::DatabaseContext; -use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; +use rayexec_execution::expr; +use rayexec_execution::functions::table::{ + try_location_and_access_config_from_args, + PlannedTableFunction, + ScanPlanner, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, +}; use rayexec_execution::functions::{FunctionInfo, Signature}; +use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; -use rayexec_execution::storage::table_storage::DataTable; -use rayexec_io::location::{AccessConfig, FileLocation}; use crate::datatable::IcebergDataTable; use crate::table::Table; @@ -39,83 +47,46 @@ impl FunctionInfo for ReadIceberg { } impl TableFunction for ReadIceberg { - fn plan_and_initialize<'a>( - &self, - _context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - let func = self.clone(); - Box::pin(async move { ReadIcebergImpl::initialize(func, args).await }) - } - - fn decode_state(&self, _state: &[u8]) -> Result> { - // TODO - not_implemented!("decode iceberg state") + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::Scan(self) } } -#[derive(Debug, Clone)] -struct ReadIcebergState { - _location: FileLocation, - _conf: AccessConfig, - schema: Schema, - table: Option>, // Populate on re-init if needed. -} - -#[derive(Debug, Clone)] -pub struct ReadIcebergImpl { - func: ReadIceberg, - state: ReadIcebergState, +impl ScanPlanner for ReadIceberg { + fn plan<'a>( + &self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result> { + Self::plan_inner(self.clone(), context, positional_inputs, named_inputs).boxed() + } } -impl ReadIcebergImpl { - async fn initialize( - func: ReadIceberg, - args: TableFunctionInputs, - ) -> Result> { - let (location, conf) = args.try_location_and_access_config()?; - let provider = func.runtime.file_provider(); +impl ReadIceberg { + async fn plan_inner<'a>( + self: Self, + _context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { + let (location, conf) = + try_location_and_access_config_from_args(&self, &positional_inputs, &named_inputs)?; + let provider = self.runtime.file_provider(); // TODO: Fetch stats, use during planning. let table = Table::load(location.clone(), provider, conf.clone()).await?; let schema = table.schema()?; - Ok(Box::new(ReadIcebergImpl { - func, - state: ReadIcebergState { - _location: location, - _conf: conf, - schema, - table: Some(Arc::new(table)), - }, - })) - } -} - -impl PlannedTableFunction2 for ReadIcebergImpl { - fn reinitialize(&self) -> BoxFuture> { - // TODO: See delta - Box::pin(async move { not_implemented!("reinit iceberg state") }) - } - - fn table_function(&self) -> &dyn TableFunction { - &self.func - } - - fn encode_state(&self, _state: &mut Vec) -> Result<()> { - not_implemented!("encode iceberg state") - } - - fn schema(&self) -> Schema { - self.state.schema.clone() - } - - fn datatable(&self) -> Result> { - let table = match self.state.table.as_ref() { - Some(table) => table.clone(), - None => return Err(RayexecError::new("Iceberg table not initialized")), - }; - - Ok(Box::new(IcebergDataTable { table })) + Ok(PlannedTableFunction { + function: Box::new(self), + positional_inputs: positional_inputs.into_iter().map(expr::lit).collect(), + named_inputs, + function_impl: TableFunctionImpl::Scan(Arc::new(IcebergDataTable { + table: Arc::new(table), // TODO: Arc Arc + })), + cardinality: StatisticsValue::Unknown, + schema, + }) } } diff --git a/crates/rayexec_unity_catalog/src/functions.rs b/crates/rayexec_unity_catalog/src/functions.rs index f3669825c..0dc25fcea 100644 --- a/crates/rayexec_unity_catalog/src/functions.rs +++ b/crates/rayexec_unity_catalog/src/functions.rs @@ -1,18 +1,29 @@ +use std::collections::HashMap; use std::fmt::{self, Debug}; use std::marker::PhantomData; +use std::sync::Arc; use futures::future::BoxFuture; use futures::stream::BoxStream; -use futures::TryStreamExt; +use futures::{FutureExt, TryStreamExt}; use rayexec_bullet::array::Array; use rayexec_bullet::batch::Batch; use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_bullet::field::{Field, Schema}; -use rayexec_error::{not_implemented, Result}; +use rayexec_bullet::scalar::OwnedScalarValue; +use rayexec_error::Result; use rayexec_execution::database::DatabaseContext; -use rayexec_execution::functions::table::inputs::TableFunctionInputs; -use rayexec_execution::functions::table::{PlannedTableFunction2, TableFunction}; +use rayexec_execution::expr; +use rayexec_execution::functions::table::{ + try_get_positional, + PlannedTableFunction, + ScanPlanner, + TableFunction, + TableFunctionImpl, + TableFunctionPlanner, +}; use rayexec_execution::functions::{FunctionInfo, Signature}; +use rayexec_execution::logical::statistics::StatisticsValue; use rayexec_execution::runtime::Runtime; use rayexec_execution::storage::table_storage::{ DataTable, @@ -43,9 +54,10 @@ pub trait UnityObjectsOperation: /// Create the connection state. fn create_connection_state( - runtime: R, + info: UnityObjects, context: &DatabaseContext, - args: TableFunctionInputs, + positional_args: Vec, + named_args: HashMap, ) -> BoxFuture<'_, Result>; /// Create a stream state from the connection state. @@ -95,15 +107,16 @@ impl UnityObjectsOperation for ListSchemasOperation { } fn create_connection_state( - runtime: R, + info: UnityObjects, _context: &DatabaseContext, - args: TableFunctionInputs, + positional_args: Vec, + _named_args: HashMap, ) -> BoxFuture<'_, Result> { Box::pin(async move { - let endpoint = args.try_get_position(0)?.try_as_str()?; - let catalog = args.try_get_position(1)?.try_as_str()?; + let endpoint = try_get_positional(&info, 0, &positional_args)?.try_as_str()?; + let catalog = try_get_positional(&info, 1, &positional_args)?.try_as_str()?; - let conn = UnityCatalogConnection::connect(runtime, endpoint, catalog).await?; + let conn = UnityCatalogConnection::connect(info.runtime, endpoint, catalog).await?; Ok(ListSchemasConnectionState { conn }) }) @@ -177,16 +190,17 @@ impl UnityObjectsOperation for ListTablesOperation { } fn create_connection_state( - runtime: R, + info: UnityObjects, _context: &DatabaseContext, - args: TableFunctionInputs, + positional_args: Vec, + _named_args: HashMap, ) -> BoxFuture<'_, Result> { Box::pin(async move { - let endpoint = args.try_get_position(0)?.try_as_str()?; - let catalog = args.try_get_position(1)?.try_as_str()?; - let schema = args.try_get_position(2)?.try_as_str()?; + let endpoint = try_get_positional(&info, 0, &positional_args)?.try_as_str()?; + let catalog = try_get_positional(&info, 1, &positional_args)?.try_as_str()?; + let schema = try_get_positional(&info, 2, &positional_args)?.try_as_str()?; - let conn = UnityCatalogConnection::connect(runtime, endpoint, catalog).await?; + let conn = UnityCatalogConnection::connect(info.runtime, endpoint, catalog).await?; Ok(ListTablesConnectionState { conn, @@ -262,48 +276,48 @@ impl> FunctionInfo for UnityObjects> TableFunction for UnityObjects { - fn plan_and_initialize<'a>( - &self, - context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - let func = self.clone(); - let runtime = self.runtime.clone(); - - Box::pin(async move { - let state = O::create_connection_state(runtime, context, args).await?; - Ok(Box::new(UnityObjectsImpl:: { func, state }) as _) - }) + fn planner(&self) -> TableFunctionPlanner { + TableFunctionPlanner::Scan(self) } - - fn decode_state(&self, _state: &[u8]) -> Result> { - not_implemented!("decode state for unity operation") - } -} - -#[derive(Debug, Clone)] -pub struct UnityObjectsImpl> { - func: UnityObjects, - state: O::ConnectionState, } -impl> PlannedTableFunction2 for UnityObjectsImpl { - fn table_function(&self) -> &dyn TableFunction { - &self.func - } - - fn schema(&self) -> Schema { - O::schema() - } - - fn encode_state(&self, _state: &mut Vec) -> Result<()> { - not_implemented!("decode state for unity operation") +impl> ScanPlanner for UnityObjects { + fn plan<'a>( + &self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> BoxFuture<'a, Result> { + Self::plan_inner(self.clone(), context, positional_inputs, named_inputs).boxed() } +} - fn datatable(&self) -> Result> { - Ok(Box::new(UnityObjectsDataTable:: { - state: self.state.clone(), - })) +impl> UnityObjects { + async fn plan_inner<'a>( + self: Self, + context: &'a DatabaseContext, + positional_inputs: Vec, + named_inputs: HashMap, + ) -> Result { + // TODO: Remove clones. + let state = O::create_connection_state( + self.clone(), + context, + positional_inputs.clone(), + named_inputs.clone(), + ) + .await?; + + Ok(PlannedTableFunction { + function: Box::new(self), + positional_inputs: positional_inputs.into_iter().map(expr::lit).collect(), + named_inputs, + function_impl: TableFunctionImpl::Scan(Arc::new(UnityObjectsDataTable:: { + state, + })), + cardinality: StatisticsValue::Unknown, + schema: O::schema(), + }) } } From 6d893732aadd409e08b58d0619adc3de0466e7cb Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 16:30:48 -0600 Subject: [PATCH 20/23] lint --- crates/rayexec_execution/src/functions/mod.rs | 1 - .../rayexec_execution/src/functions/proto.rs | 77 ++++--------- .../src/functions/scalar/mod.rs | 1 - .../src/functions/table/builtin/mod.rs | 1 - .../src/functions/table/builtin/refresh.rs | 101 ------------------ .../src/functions/table/builtin/series.rs | 2 +- .../src/functions/table/inputs.rs | 80 -------------- .../src/functions/table/mod.rs | 91 +--------------- .../src/logical/resolver/expr_resolver.rs | 1 - .../src/logical/resolver/mod.rs | 3 +- .../src/logical/resolver/resolve_hybrid.rs | 2 - .../resolver/resolved_table_function.rs | 6 +- crates/rayexec_parser/src/meta.rs | 2 +- 13 files changed, 32 insertions(+), 336 deletions(-) delete mode 100644 crates/rayexec_execution/src/functions/table/builtin/refresh.rs delete mode 100644 crates/rayexec_execution/src/functions/table/inputs.rs diff --git a/crates/rayexec_execution/src/functions/mod.rs b/crates/rayexec_execution/src/functions/mod.rs index a2a6941ea..83b4fe22d 100644 --- a/crates/rayexec_execution/src/functions/mod.rs +++ b/crates/rayexec_execution/src/functions/mod.rs @@ -12,7 +12,6 @@ use fmtutil::IntoDisplayableSlice; use implicit::{implicit_cast_score, NO_CAST_SCORE}; use rayexec_bullet::datatype::{DataType, DataTypeId}; use rayexec_error::{RayexecError, Result}; -use scalar::ScalarFunction; /// Function signature. // TODO: Include named args. diff --git a/crates/rayexec_execution/src/functions/proto.rs b/crates/rayexec_execution/src/functions/proto.rs index 6bd0c8a5a..37a156f0c 100644 --- a/crates/rayexec_execution/src/functions/proto.rs +++ b/crates/rayexec_execution/src/functions/proto.rs @@ -7,8 +7,7 @@ use rayexec_proto::ProtoConv; use super::aggregate::{AggregateFunction, PlannedAggregateFunction}; use super::copy::{CopyToArgs, CopyToFunction}; use super::scalar::{PlannedScalarFunction, ScalarFunction}; -use super::table::inputs::TableFunctionInputs; -use super::table::{PlannedTableFunction2, TableFunction}; +use super::table::{PlannedTableFunction, TableFunction}; use crate::database::catalog::CatalogTx; use crate::database::DatabaseContext; use crate::proto::DatabaseProtoConv; @@ -146,68 +145,34 @@ impl DatabaseProtoConv for Box { } } -impl DatabaseProtoConv for Box { +impl DatabaseProtoConv for PlannedTableFunction { type ProtoType = rayexec_proto::generated::functions::PlannedTableFunction; fn to_proto_ctx(&self, _context: &DatabaseContext) -> Result { - let mut state = Vec::new(); - self.encode_state(&mut state)?; - - Ok(Self::ProtoType { - name: self.table_function().name().to_string(), - state, - }) - } - - fn from_proto_ctx(proto: Self::ProtoType, context: &DatabaseContext) -> Result { - let tx = &CatalogTx {}; - let ent = context - .system_catalog()? - .get_schema(tx, FUNCTION_LOOKUP_CATALOG)? - .required("lookup schema")? - .get_table_function(tx, &proto.name)? - .required("table function")?; - let ent = ent.try_as_table_function_entry()?; - - let planned = ent.function.decode_state(&proto.state)?; + unimplemented!() + // let mut state = Vec::new(); + // self.encode_state(&mut state)?; - Ok(planned) + // Ok(Self::ProtoType { + // name: self.table_function().name().to_string(), + // state, + // }) } -} - -impl ProtoConv for TableFunctionInputs { - type ProtoType = rayexec_proto::generated::functions::TableFunctionArgs; - fn to_proto(&self) -> Result { - let mut named = HashMap::new(); - for (key, val) in &self.named { - named.insert(key.clone(), val.to_proto()?); - } - - Ok(Self::ProtoType { - named, - positional: self - .positional - .iter() - .map(|v| v.to_proto()) - .collect::>>()?, - }) - } + fn from_proto_ctx(_proto: Self::ProtoType, _context: &DatabaseContext) -> Result { + unimplemented!() + // let tx = &CatalogTx {}; + // let ent = context + // .system_catalog()? + // .get_schema(tx, FUNCTION_LOOKUP_CATALOG)? + // .required("lookup schema")? + // .get_table_function(tx, &proto.name)? + // .required("table function")?; + // let ent = ent.try_as_table_function_entry()?; - fn from_proto(proto: Self::ProtoType) -> Result { - let mut named = HashMap::new(); - for (key, val) in proto.named { - named.insert(key, OwnedScalarValue::from_proto(val)?); - } + // let planned = ent.function.decode_state(&proto.state)?; - Ok(Self { - named, - positional: proto - .positional - .into_iter() - .map(OwnedScalarValue::from_proto) - .collect::>>()?, - }) + // Ok(planned) } } diff --git a/crates/rayexec_execution/src/functions/scalar/mod.rs b/crates/rayexec_execution/src/functions/scalar/mod.rs index e53158374..94251627c 100644 --- a/crates/rayexec_execution/src/functions/scalar/mod.rs +++ b/crates/rayexec_execution/src/functions/scalar/mod.rs @@ -1,6 +1,5 @@ pub mod builtin; -use std::any::Any; use std::fmt::Debug; use std::hash::Hash; diff --git a/crates/rayexec_execution/src/functions/table/builtin/mod.rs b/crates/rayexec_execution/src/functions/table/builtin/mod.rs index 4b0c05377..fcd1cf122 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/mod.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/mod.rs @@ -1,4 +1,3 @@ -pub mod refresh; pub mod series; pub mod system; diff --git a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs b/crates/rayexec_execution/src/functions/table/builtin/refresh.rs deleted file mode 100644 index fa750739d..000000000 --- a/crates/rayexec_execution/src/functions/table/builtin/refresh.rs +++ /dev/null @@ -1,101 +0,0 @@ -use std::fmt::Debug; -use std::marker::PhantomData; -use std::sync::Arc; - -use futures::future::BoxFuture; -use rayexec_bullet::datatype::{DataType, DataTypeId}; -use rayexec_bullet::field::{Field, Schema}; -use rayexec_error::{RayexecError, Result}; - -use crate::database::memory_catalog::MemoryCatalog; -use crate::database::DatabaseContext; -use crate::functions::table::{TableFunction, TableFunctionInputs, TableFunctionPlanner}; -use crate::functions::{FunctionInfo, Signature}; -use crate::storage::catalog_storage::CatalogStorage; - -pub trait RefreshOperation: Debug + Clone + Copy + PartialEq + Eq + Sync + Send + 'static { - const NAME: &'static str; - type State: Debug + Clone + Sync + Send; - - fn schema() -> Schema; - - #[allow(dead_code)] - fn create_state(context: &DatabaseContext, args: TableFunctionInputs) -> Result; - - #[allow(dead_code)] - fn refresh(state: &Self::State) -> Result>>; -} - -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub struct RefreshSchemas; - -#[derive(Debug, Clone)] -pub struct RefreshSchemasState { - catalog: Arc, - catalog_storage: Arc, -} - -impl RefreshOperation for RefreshSchemas { - const NAME: &'static str = "refresh_schemas"; - - type State = RefreshSchemasState; - - fn schema() -> Schema { - Schema::new([Field::new("count", DataType::Int64, false)]) - } - - fn create_state(context: &DatabaseContext, args: TableFunctionInputs) -> Result { - let database_name = args.try_get_position(0)?.try_as_str()?; - - let database = context.get_database(database_name)?; - let catalog_storage = database.catalog_storage.as_ref().ok_or_else(|| { - RayexecError::new(format!("Missing catalog storage for '{database_name}'")) - })?; - - Ok(RefreshSchemasState { - catalog: database.catalog.clone(), - catalog_storage: catalog_storage.clone(), - }) - } - - fn refresh(state: &Self::State) -> Result>> { - state.catalog_storage.load_schemas(&state.catalog) - } -} - -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub struct RefreshObjects { - _op: PhantomData, -} - -impl RefreshObjects { - pub const fn new() -> Self { - RefreshObjects { _op: PhantomData } - } -} - -impl Default for RefreshObjects { - fn default() -> Self { - Self::new() - } -} - -impl FunctionInfo for RefreshObjects { - fn name(&self) -> &'static str { - O::NAME - } - - fn signatures(&self) -> &[Signature] { - &[Signature { - positional_args: &[], - variadic_arg: None, - return_type: DataTypeId::Any, - }] - } -} - -impl TableFunction for RefreshObjects { - fn planner(&self) -> TableFunctionPlanner { - unimplemented!() - } -} diff --git a/crates/rayexec_execution/src/functions/table/builtin/series.rs b/crates/rayexec_execution/src/functions/table/builtin/series.rs index 556fe5536..bf5a3688b 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/series.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/series.rs @@ -11,7 +11,7 @@ use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_bullet::storage::PrimitiveStorage; use rayexec_error::{RayexecError, Result}; -use crate::execution::operators::{PollFinalize, PollPull, PollPush}; +use crate::execution::operators::{PollFinalize, PollPush}; use crate::expr::{self, Expression}; use crate::functions::table::inout::{InOutPollPull, TableInOutFunction, TableInOutPartitionState}; use crate::functions::table::{ diff --git a/crates/rayexec_execution/src/functions/table/inputs.rs b/crates/rayexec_execution/src/functions/table/inputs.rs deleted file mode 100644 index 959cd2150..000000000 --- a/crates/rayexec_execution/src/functions/table/inputs.rs +++ /dev/null @@ -1,80 +0,0 @@ -use std::collections::HashMap; -use std::fmt::Debug; - -use rayexec_bullet::scalar::OwnedScalarValue; -use rayexec_error::{RayexecError, Result}; -use rayexec_io::location::{AccessConfig, FileLocation}; -use rayexec_io::s3::credentials::AwsCredentials; -use rayexec_io::s3::S3Location; -use serde::{Deserialize, Serialize}; - -use super::TableFunction; - -#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] -pub struct TableFunctionInputs { - /// Named arguments to a table function. - pub named: HashMap, - - /// Positional arguments to a table function. - pub positional: Vec, -} - -impl TableFunctionInputs { - /// Try to get a file location and access config from the table args. - // TODO: Secrets provider that we pass in allowing us to get creds from some - // secrets store. - pub fn try_location_and_access_config(&self) -> Result<(FileLocation, AccessConfig)> { - let loc = match self.positional.first() { - Some(loc) => { - let loc = loc.try_as_str()?; - FileLocation::parse(loc) - } - None => return Err(RayexecError::new("Expected at least one position argument")), - }; - - let conf = match &loc { - FileLocation::Url(url) => { - if S3Location::is_s3_location(url) { - let key_id = self.try_get_named("key_id")?.try_as_str()?.to_string(); - let secret = self.try_get_named("secret")?.try_as_str()?.to_string(); - let region = self.try_get_named("region")?.try_as_str()?.to_string(); - - AccessConfig::S3 { - credentials: AwsCredentials { key_id, secret }, - region, - } - } else { - AccessConfig::None - } - } - FileLocation::Path(_) => AccessConfig::None, - }; - - Ok((loc, conf)) - } - - pub fn try_get_named(&self, name: &str) -> Result<&OwnedScalarValue> { - self.named - .get(name) - .ok_or_else(|| RayexecError::new(format!("Expected named argument '{name}'"))) - } - - pub fn try_get_position(&self, pos: usize) -> Result<&OwnedScalarValue> { - self.positional - .get(pos) - .ok_or_else(|| RayexecError::new(format!("Expected argument at position {pos}"))) - } -} - -pub fn check_named_args_is_empty( - func: &dyn TableFunction, - args: &TableFunctionInputs, -) -> Result<()> { - if !args.named.is_empty() { - return Err(RayexecError::new(format!( - "'{}' does not take named arguments", - func.name() - ))); - } - Ok(()) -} diff --git a/crates/rayexec_execution/src/functions/table/mod.rs b/crates/rayexec_execution/src/functions/table/mod.rs index b5b15e49b..9e449fc29 100644 --- a/crates/rayexec_execution/src/functions/table/mod.rs +++ b/crates/rayexec_execution/src/functions/table/mod.rs @@ -1,6 +1,5 @@ pub mod builtin; pub mod inout; -pub mod inputs; use std::collections::HashMap; use std::fmt::Debug; @@ -8,9 +7,7 @@ use std::sync::Arc; use dyn_clone::DynClone; use futures::future::BoxFuture; -use futures::FutureExt; use inout::TableInOutFunction; -use inputs::TableFunctionInputs; use rayexec_bullet::field::Schema; use rayexec_bullet::scalar::OwnedScalarValue; use rayexec_error::{RayexecError, Result}; @@ -34,35 +31,8 @@ use crate::storage::table_storage::DataTable; /// /// The specialized variant should be determined by function argument inputs. pub trait TableFunction: FunctionInfo + Debug + Sync + Send + DynClone { - /// Plan the table function using the provide args, and do any necessary - /// initialization. - /// - /// Intialization may include opening connections a remote database, and - /// should be used determine the schema of the table we'll be returning. Any - /// connections should remain open through execution. - fn plan_and_initialize<'a>( - &self, - context: &'a DatabaseContext, - args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - unimplemented!() - } - - fn initialize<'a>( - &self, - _context: &'a DatabaseContext, - _args: TableFunctionInputs, - ) -> BoxFuture<'a, Result>> { - unimplemented!() - } - - fn decode_state(&self, state: &[u8]) -> Result> { - unimplemented!() - } - - fn planner(&self) -> TableFunctionPlanner { - unimplemented!() - } + /// Return a planner that will produce a planned table function. + fn planner(&self) -> TableFunctionPlanner; } impl Clone for Box { @@ -85,9 +55,12 @@ impl PartialEq for dyn TableFunction + '_ { impl Eq for dyn TableFunction {} +/// The types of table function planners supported. #[derive(Debug)] pub enum TableFunctionPlanner<'a> { + /// Produces a table function that accept inputs and produce outputs. InOut(&'a dyn InOutPlanner), + /// Produces a table function that acts as a just a scan. Scan(&'a dyn ScanPlanner), } @@ -154,60 +127,6 @@ pub enum TableFunctionImpl { InOut(Box), } -pub trait PlannedTableFunction2: Debug + Sync + Send + DynClone { - /// Reinitialize the table function, including re-opening any connections - /// needed. - /// - /// This is called immediately after deserializing a planned function in - /// order populate fields that cannot be serialized and moved across - /// machines. - /// - /// The default implementation does nothing. - fn reinitialize(&self) -> BoxFuture> { - async move { Ok(()) }.boxed() - } - - fn encode_state(&self, state: &mut Vec) -> Result<()>; - - /// Returns a reference to the table function that initialized this - /// function. - fn table_function(&self) -> &dyn TableFunction; - - /// Get the schema for the function output. - fn schema(&self) -> Schema; - - /// Get the cardinality of the output. - fn cardinality(&self) -> StatisticsValue { - StatisticsValue::Unknown - } - - /// Return a data table representing the function output. - /// - /// An engine runtime is provided for table funcs that return truly async - /// data tables. - fn datatable(&self) -> Result>; -} - -impl PartialEq for Box { - fn eq(&self, other: &dyn PlannedTableFunction2) -> bool { - self.as_ref() == other - } -} - -impl PartialEq for dyn PlannedTableFunction2 + '_ { - fn eq(&self, other: &dyn PlannedTableFunction2) -> bool { - self.table_function() == other.table_function() && self.schema() == other.schema() - } -} - -impl Eq for dyn PlannedTableFunction2 {} - -impl Clone for Box { - fn clone(&self) -> Self { - dyn_clone::clone_box(&**self) - } -} - /// Try to get a file location and access config from the table args. // TODO: Secrets provider that we pass in allowing us to get creds from some // secrets store. diff --git a/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs b/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs index b04177983..70ae225c5 100644 --- a/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs +++ b/crates/rayexec_execution/src/logical/resolver/expr_resolver.rs @@ -9,7 +9,6 @@ use super::resolved_function::{ResolvedFunction, SpecialBuiltinFunction}; use super::resolved_table_function::ConstantFunctionArgs; use super::{ResolveContext, ResolvedMeta, Resolver}; use crate::database::catalog_entry::CatalogEntryType; -use crate::functions::table::inputs::TableFunctionInputs; use crate::logical::binder::expr_binder::BaseExpressionBinder; use crate::logical::operator::LocationRequirement; diff --git a/crates/rayexec_execution/src/logical/resolver/mod.rs b/crates/rayexec_execution/src/logical/resolver/mod.rs index 57bdd819f..75208f435 100644 --- a/crates/rayexec_execution/src/logical/resolver/mod.rs +++ b/crates/rayexec_execution/src/logical/resolver/mod.rs @@ -16,7 +16,7 @@ use rayexec_bullet::scalar::decimal::{Decimal128Type, Decimal64Type, DecimalType use rayexec_bullet::scalar::{OwnedScalarValue, ScalarValue}; use rayexec_error::{OptionExt, RayexecError, Result}; use rayexec_io::location::FileLocation; -use rayexec_parser::ast::{self, ColumnDef, FunctionArg, ObjectReference}; +use rayexec_parser::ast::{self, ColumnDef, ObjectReference}; use rayexec_parser::meta::{AstMeta, Raw}; use rayexec_parser::parser; use rayexec_parser::statement::{RawStatement, Statement}; @@ -43,7 +43,6 @@ use crate::database::DatabaseContext; use crate::datasource::FileHandlers; use crate::functions::copy::CopyToArgs; use crate::functions::proto::FUNCTION_LOOKUP_CATALOG; -use crate::functions::table::inputs::TableFunctionInputs; use crate::functions::table::TableFunctionPlanner; use crate::logical::operator::LocationRequirement; diff --git a/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs b/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs index 1afed9ae7..d4cfaf1a0 100644 --- a/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs +++ b/crates/rayexec_execution/src/logical/resolver/resolve_hybrid.rs @@ -1,4 +1,3 @@ -use std::collections::HashMap; use std::sync::Arc; use rayexec_error::{RayexecError, Result}; @@ -13,7 +12,6 @@ use crate::database::memory_catalog::MemoryCatalog; use crate::database::{Database, DatabaseContext}; use crate::datasource::{DataSourceRegistry, FileHandlers}; use crate::functions::table::TableFunctionPlanner; -use crate::logical::binder::constant_binder::ConstantBinder; use crate::logical::operator::LocationRequirement; use crate::logical::resolver::ResolveMode; diff --git a/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs b/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs index 2dab392e7..295d383d1 100644 --- a/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs +++ b/crates/rayexec_execution/src/logical/resolver/resolved_table_function.rs @@ -38,7 +38,7 @@ impl ResolvedTableFunctionReference { impl DatabaseProtoConv for ResolvedTableFunctionReference { type ProtoType = rayexec_proto::generated::resolver::ResolvedTableFunctionReference; - fn to_proto_ctx(&self, context: &DatabaseContext) -> Result { + fn to_proto_ctx(&self, _context: &DatabaseContext) -> Result { unimplemented!() // Ok(Self::ProtoType { // name: self.name.clone(), @@ -46,7 +46,7 @@ impl DatabaseProtoConv for ResolvedTableFunctionReference { // }) } - fn from_proto_ctx(proto: Self::ProtoType, context: &DatabaseContext) -> Result { + fn from_proto_ctx(_proto: Self::ProtoType, _context: &DatabaseContext) -> Result { unimplemented!() // Ok(Self { // name: proto.name, @@ -82,7 +82,7 @@ impl ProtoConv for UnresolvedTableFunctionReference { // }) } - fn from_proto(proto: Self::ProtoType) -> Result { + fn from_proto(_proto: Self::ProtoType) -> Result { unimplemented!() // Ok(Self { // reference: ast::ObjectReference::from_proto(proto.reference.required("reference")?)?, diff --git a/crates/rayexec_parser/src/meta.rs b/crates/rayexec_parser/src/meta.rs index b71ead3b0..8c9a4b72a 100644 --- a/crates/rayexec_parser/src/meta.rs +++ b/crates/rayexec_parser/src/meta.rs @@ -3,7 +3,7 @@ use std::fmt::Debug; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; -use crate::ast::{CopyOption, CopyToTarget, DataType, FunctionArg, ObjectReference, ShowReference}; +use crate::ast::{CopyOption, CopyToTarget, DataType, ObjectReference, ShowReference}; /// Metadata associated with sql statements. /// From 4b21daff05a4f8e061e1210eaa3894c07ec5541a Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 16:36:10 -0600 Subject: [PATCH 21/23] more lint --- .../rayexec_execution/src/execution/operators/table_inout.rs | 4 +--- .../rayexec_execution/src/functions/scalar/builtin/negate.rs | 2 +- .../rayexec_execution/src/functions/table/builtin/series.rs | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/crates/rayexec_execution/src/execution/operators/table_inout.rs b/crates/rayexec_execution/src/execution/operators/table_inout.rs index bdd735d04..22ab4a1f6 100644 --- a/crates/rayexec_execution/src/execution/operators/table_inout.rs +++ b/crates/rayexec_execution/src/execution/operators/table_inout.rs @@ -108,9 +108,7 @@ impl ExecutableOperator for PhysicalTableInOut { // TODO: Remove needing to do this, the clones should be cheap, but the // expression execution is wasteful. match state.function_state.poll_push(cx, inputs)? { - PollPush::Pending(_) => { - return Ok(PollPush::Pending(orig)); - } + PollPush::Pending(_) => Ok(PollPush::Pending(orig)), other => { // Batch was pushed to the function state, compute additional // outputs. diff --git a/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs b/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs index 49136c40c..1b5f13c35 100644 --- a/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs +++ b/crates/rayexec_execution/src/functions/scalar/builtin/negate.rs @@ -44,7 +44,7 @@ impl FunctionInfo for Negate { Signature::new_positional(&[DataTypeId::Int128], DataTypeId::Int128), Signature::new_positional(&[DataTypeId::Interval], DataTypeId::Interval), ]; - &SIGS + SIGS } } diff --git a/crates/rayexec_execution/src/functions/table/builtin/series.rs b/crates/rayexec_execution/src/functions/table/builtin/series.rs index bf5a3688b..209363883 100644 --- a/crates/rayexec_execution/src/functions/table/builtin/series.rs +++ b/crates/rayexec_execution/src/functions/table/builtin/series.rs @@ -91,7 +91,7 @@ impl InOutPlanner for GenerateSeriesInOutPlanner { if positional_inputs.len() == 2 { // Add constant for the 'step' argument. - positional_inputs.push(expr::lit(1 as i64)) + positional_inputs.push(expr::lit(1_i64)) } Ok(PlannedTableFunction { From 92ce50d2e2e0e293224df5a6f4ef0ec315e936f1 Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 16:37:37 -0600 Subject: [PATCH 22/23] lint --- crates/rayexec_csv/src/read_csv.rs | 2 +- crates/rayexec_delta/src/read_delta.rs | 2 +- crates/rayexec_iceberg/src/read_iceberg.rs | 2 +- crates/rayexec_parquet/src/functions/read_parquet.rs | 2 +- crates/rayexec_postgres/src/read_postgres.rs | 6 +++--- crates/rayexec_unity_catalog/src/functions.rs | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/crates/rayexec_csv/src/read_csv.rs b/crates/rayexec_csv/src/read_csv.rs index 4b8d31627..e31b0ea92 100644 --- a/crates/rayexec_csv/src/read_csv.rs +++ b/crates/rayexec_csv/src/read_csv.rs @@ -67,7 +67,7 @@ impl ScanPlanner for ReadCsv { impl ReadCsv { async fn plan_inner<'a>( - self: Self, + self, _context: &'a DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, diff --git a/crates/rayexec_delta/src/read_delta.rs b/crates/rayexec_delta/src/read_delta.rs index fcd7d5d1c..c84f9c317 100644 --- a/crates/rayexec_delta/src/read_delta.rs +++ b/crates/rayexec_delta/src/read_delta.rs @@ -65,7 +65,7 @@ impl ScanPlanner for ReadDelta { impl ReadDelta { async fn plan_inner<'a>( - self: Self, + self, _context: &'a DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, diff --git a/crates/rayexec_iceberg/src/read_iceberg.rs b/crates/rayexec_iceberg/src/read_iceberg.rs index f7283c0ff..930199e2a 100644 --- a/crates/rayexec_iceberg/src/read_iceberg.rs +++ b/crates/rayexec_iceberg/src/read_iceberg.rs @@ -65,7 +65,7 @@ impl ScanPlanner for ReadIceberg { impl ReadIceberg { async fn plan_inner<'a>( - self: Self, + self, _context: &'a DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, diff --git a/crates/rayexec_parquet/src/functions/read_parquet.rs b/crates/rayexec_parquet/src/functions/read_parquet.rs index 55a598f3a..5c345c1e5 100644 --- a/crates/rayexec_parquet/src/functions/read_parquet.rs +++ b/crates/rayexec_parquet/src/functions/read_parquet.rs @@ -67,7 +67,7 @@ impl ScanPlanner for ReadParquet { impl ReadParquet { async fn plan_inner<'a>( - self: Self, + self, _context: &'a DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, diff --git a/crates/rayexec_postgres/src/read_postgres.rs b/crates/rayexec_postgres/src/read_postgres.rs index b3adf6a25..7ba817710 100644 --- a/crates/rayexec_postgres/src/read_postgres.rs +++ b/crates/rayexec_postgres/src/read_postgres.rs @@ -60,7 +60,7 @@ impl ScanPlanner for ReadPostgres { impl ReadPostgres { async fn plan_inner<'a>( - self: Self, + self, _context: &'a DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, @@ -74,13 +74,13 @@ impl ReadPostgres { return Err(RayexecError::new("read_postgres requires 3 arguments")); } - let conn_str = positional_inputs.get(0).unwrap().try_as_str()?; + let conn_str = positional_inputs.first().unwrap().try_as_str()?; let schema = positional_inputs.get(1).unwrap().try_as_str()?; let table = positional_inputs.get(2).unwrap().try_as_str()?; let client = PostgresClient::connect(conn_str, &self.runtime).await?; - let fields = match client.get_fields_and_types(&schema, &table).await? { + let fields = match client.get_fields_and_types(schema, table).await? { Some((fields, _)) => fields, None => return Err(RayexecError::new("Table not found")), }; diff --git a/crates/rayexec_unity_catalog/src/functions.rs b/crates/rayexec_unity_catalog/src/functions.rs index 0dc25fcea..2ea898f6c 100644 --- a/crates/rayexec_unity_catalog/src/functions.rs +++ b/crates/rayexec_unity_catalog/src/functions.rs @@ -294,7 +294,7 @@ impl> ScanPlanner for UnityObjects impl> UnityObjects { async fn plan_inner<'a>( - self: Self, + self, context: &'a DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, From d99d71f2dbcd461c6a399844aba0c5b7eb7762ac Mon Sep 17 00:00:00 2001 From: Sean Smith Date: Sun, 15 Dec 2024 16:40:03 -0600 Subject: [PATCH 23/23] lint --- crates/rayexec_csv/src/read_csv.rs | 4 ++-- crates/rayexec_delta/src/read_delta.rs | 4 ++-- crates/rayexec_iceberg/src/read_iceberg.rs | 4 ++-- crates/rayexec_parquet/src/functions/read_parquet.rs | 4 ++-- crates/rayexec_postgres/src/read_postgres.rs | 4 ++-- crates/rayexec_unity_catalog/src/functions.rs | 4 ++-- 6 files changed, 12 insertions(+), 12 deletions(-) diff --git a/crates/rayexec_csv/src/read_csv.rs b/crates/rayexec_csv/src/read_csv.rs index e31b0ea92..457837531 100644 --- a/crates/rayexec_csv/src/read_csv.rs +++ b/crates/rayexec_csv/src/read_csv.rs @@ -66,9 +66,9 @@ impl ScanPlanner for ReadCsv { } impl ReadCsv { - async fn plan_inner<'a>( + async fn plan_inner( self, - _context: &'a DatabaseContext, + _context: &DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, ) -> Result { diff --git a/crates/rayexec_delta/src/read_delta.rs b/crates/rayexec_delta/src/read_delta.rs index c84f9c317..20e4bcd07 100644 --- a/crates/rayexec_delta/src/read_delta.rs +++ b/crates/rayexec_delta/src/read_delta.rs @@ -64,9 +64,9 @@ impl ScanPlanner for ReadDelta { } impl ReadDelta { - async fn plan_inner<'a>( + async fn plan_inner( self, - _context: &'a DatabaseContext, + _context: &DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, ) -> Result { diff --git a/crates/rayexec_iceberg/src/read_iceberg.rs b/crates/rayexec_iceberg/src/read_iceberg.rs index 930199e2a..f88aa909b 100644 --- a/crates/rayexec_iceberg/src/read_iceberg.rs +++ b/crates/rayexec_iceberg/src/read_iceberg.rs @@ -64,9 +64,9 @@ impl ScanPlanner for ReadIceberg { } impl ReadIceberg { - async fn plan_inner<'a>( + async fn plan_inner( self, - _context: &'a DatabaseContext, + _context: &DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, ) -> Result { diff --git a/crates/rayexec_parquet/src/functions/read_parquet.rs b/crates/rayexec_parquet/src/functions/read_parquet.rs index 5c345c1e5..54c6b8f3d 100644 --- a/crates/rayexec_parquet/src/functions/read_parquet.rs +++ b/crates/rayexec_parquet/src/functions/read_parquet.rs @@ -66,9 +66,9 @@ impl ScanPlanner for ReadParquet { } impl ReadParquet { - async fn plan_inner<'a>( + async fn plan_inner( self, - _context: &'a DatabaseContext, + _context: &DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, ) -> Result { diff --git a/crates/rayexec_postgres/src/read_postgres.rs b/crates/rayexec_postgres/src/read_postgres.rs index 7ba817710..934b9a08e 100644 --- a/crates/rayexec_postgres/src/read_postgres.rs +++ b/crates/rayexec_postgres/src/read_postgres.rs @@ -59,9 +59,9 @@ impl ScanPlanner for ReadPostgres { } impl ReadPostgres { - async fn plan_inner<'a>( + async fn plan_inner( self, - _context: &'a DatabaseContext, + _context: &DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, ) -> Result { diff --git a/crates/rayexec_unity_catalog/src/functions.rs b/crates/rayexec_unity_catalog/src/functions.rs index 2ea898f6c..cde919491 100644 --- a/crates/rayexec_unity_catalog/src/functions.rs +++ b/crates/rayexec_unity_catalog/src/functions.rs @@ -293,9 +293,9 @@ impl> ScanPlanner for UnityObjects } impl> UnityObjects { - async fn plan_inner<'a>( + async fn plan_inner( self, - context: &'a DatabaseContext, + context: &DatabaseContext, positional_inputs: Vec, named_inputs: HashMap, ) -> Result {