Skip to content

Conversation

@gstvg
Copy link
Contributor

@gstvg gstvg commented Nov 25, 2025

Closes #14205

This PR adds support for lambdas with column capture and the array_transform scalar function which is used to test the lambda implementation. The changes are extensive, across various parts of the codebase, mostly tree traversals. This text aims to justify those changes, and show alternatives which may require less changes although not without trade-offs, so we can decide whether to move this forward or not, and if so, with what approach. For those who want to take a look at the code, don't waste time in the second commit as it is just adding a new field to a struct. There's a build of the documentation of this branch available online

Example array_transform usage:

array_transform([1, 2], v -> v*2)
[2, 4]
Lambda Logical and Physical Representation
pub struct Lambda {
    pub params: Vec<String>, // in the example above, vec!["v"]
    pub body: Box<Expr>, // in the example above, v*2
}

pub struct LambdaExpr {
    params: Vec<String>, // in the example above, vec!["v"]
    body: Arc<dyn PhysicalExpr>, // in the example above, v * 2
}

Changes in ScalarUDF[Impl] to support lambdas

Since lambda parameters are defined by the UDF implementation, datafusion doesn't know the type, nullability nor its metadata. So, we add a method to ScalarUDF[Impl], where the implementation returns a Field for each parameter supported for each of its lambdas

ScalarUDF[Impl] lambdas_parameters method
struct/trait ScalarUDF[Impl] {
    /// Returns the parameters that any lambda supports
    fn lambdas_parameters(
        &self,
        args: &[ValueOrLambdaParameter],
    ) -> Result<Vec<Option<Vec<Field>>>> {
        Ok(vec![None; args.len()])
    }
}

pub enum ValueOrLambdaParameter<'a> {
    /// A columnar value with the given field
    Value(FieldRef),
    /// A lambda with the given parameters names and a flag indicating whether it captures any columns
    Lambda(&'a [String], bool),
}
ArrayTransform lambdas_parameters implementation
impl ScalarUDFImpl for ArrayTransform {
    fn lambdas_parameters(
        &self,
        args: &[ValueOrLambdaParameter],
    ) -> Result<Vec<Option<Vec<Field>>>> {
        let [ValueOrLambdaParameter::Value(list), ValueOrLambdaParameter::Lambda(_, _)] =
            args
        else {
            return exec_err!(
                "{} expects a value followed by a lambda, got {:?}",
                self.name(),
                args
            );
        };

        let (field, index_type) = match list.data_type() {
            DataType::List(field) => (field, DataType::Int32),
            DataType::LargeList(field) => (field, DataType::Int64),
            DataType::FixedSizeList(field, _) => (field, DataType::Int32),
            _ => return exec_err!("expected list, got {list}"),
        };

        // we don't need to omit the index in the case the lambda don't specify, e.g. array_transform([], v -> v*2),
        // nor check whether the lambda contains more than two parameters, e.g. array_transform([], (v, i, j) -> v+i+j),
        // as datafusion will do that for us
        let value = Field::new("value", field.data_type().clone(), field.is_nullable())
            .with_metadata(field.metadata().clone());
        let index = Field::new("index", index_type, false);

        Ok(vec![None, Some(vec![value, index])])
    }
}

In order for the UDF to be able to compute its return field, it usually needs to know the return field of its lambdas, which is the case for array_transform. Because lambdas may capture columns, to compute the return field of a lambda, it's necessary to know the Field of the captured columns, which is only available in the schema and it's not passed as argument to return_fields[_from_args]. To avoid changing that, we internally use the fields returned in the newly added method ScalarUDFImpl::lambdas_parameters paired with the schema to compute the return field of the lambdas, and pass them in ReturnFieldArgs.arg_fields. We also add a slice of bools indicating if the arg in the position `i` is a lambda or not. Finally, we add a helper method to_lambda_args which merges the data in arg_fields and lambdas into a vec of ValueOrField enums allowing convenient pattern matching instead of having to inspect both arg_fields and lambdas.
ReturnFieldArgs changes
pub struct ReturnFieldArgs<'a> {
    /// The data types of the arguments to the function
    ///
    /// If argument `i` to the function is a lambda, it will be the return field of the
    /// lambda expression when evaluated with the arguments returned from  
    /// ScalarUDFImpl::lambdas_parameters`
    ///
    /// For example, with `array_transform([1], v -> v == 5)`
    /// this field will be `[Field::new("", DataType::List(DataType::Int32), false), Field::new("", DataType::Boolean, false)]`
    pub arg_fields: &'a [FieldRef],
    ... skipped fields
    /// Is argument `i` to the function a lambda?
    ///
    /// For example, with `array_transform([1], v -> v == 5)`
    /// this field will be `[false, true]`
    pub lambdas: &'a [bool],
}

/// A tagged Field indicating whether it correspond to a value or a lambda argument
#[derive(Debug)]
pub enum ValueOrLambdaField<'a> {
    /// The Field of a ColumnarValue argument
    Value(&'a FieldRef),
    /// The Field of the return of the lambda body when evaluated with the parameters from ScalarUDF::lambda_parameters
    Lambda(&'a FieldRef),
}

impl<'a> ReturnFieldArgs<'a> {
    /// Based on self.lambdas, encodes self.arg_fields to tagged enums
    /// indicating whether it correspond to a value or a lambda argument
    pub fn to_lambda_args(&self) -> Vec<ValueOrLambdaField<'a>> {
        std::iter::zip(self.arg_fields, self.lambdas)
            .map(|(field, is_lambda)| {
                if *is_lambda {
                    ValueOrLambdaField::Lambda(field)
                } else {
                    ValueOrLambdaField::Value(field)
                }
            })
            .collect()
    }
}
ArrayTransform return_field_from_args implementation
impl ScalarUDFImpl for ArrayTransform {
    fn return_field_from_args(
        &self,
        args: datafusion_expr::ReturnFieldArgs,
    ) -> Result<Arc<Field>> {
        let args = args.to_lambda_args();

        let [ValueOrLambdaField::Value(list), ValueOrLambdaField::Lambda(lambda)] =
            take_function_args(self.name(), &args)?
        else {
            return exec_err!(
                "{} expects a value followed by a lambda, got {:?}",
                self.name(),
                args
            );
        };

        // lambda is the resulting field of executing the lambda body
        // with the parameters returned in lambdas_parameters
        let field = Arc::new(Field::new(
            Field::LIST_FIELD_DEFAULT_NAME,
            lambda.data_type().clone(),
            lambda.is_nullable(),
        ));

        let return_type = match list.data_type() {
            DataType::List(_) => DataType::List(field),
            DataType::LargeList(_) => DataType::LargeList(field),
            DataType::FixedSizeList(_, size) => DataType::FixedSizeList(field, *size),
            _ => exec_err!("expected list, got {list}")?,
        };

        Ok(Arc::new(Field::new("", return_type, list.is_nullable())))
    }
}

For execution, we add the lambdas fields to ScalarFunctionArgs and the helper method to_lambda_args, similar to the changes in ReturnFieldArgs and its to_lambda_args:
ScalarFunctionArgs changes
pub struct ScalarFunctionArgs {
    /// The evaluated arguments to the function
    /// If it's a lambda, will be `ColumnarValue::Scalar(ScalarValue::Null)`
    ///
    /// For example, with `array_transform([1], v -> v == 5)`
    /// this field will be `[ColumnarValue::Scalar(ScalarValue::List([1])), ColumnarValue::Scalar(ScalarValue::Null)]`
    pub args: Vec<ColumnarValue>,
    /// Field associated with each arg, if it exists
    pub arg_fields: Vec<FieldRef>,
    ....
    /// The lambdas passed to the function
    /// If it's not a lambda it will be `None`
    ///
    /// For example, with `array_transform([1], v -> v == 5)`
    /// this field will be `[None, Some(...)]`
    pub lambdas: Option<Vec<Option<ScalarFunctionLambdaArg>>>,
}

/// A lambda argument to a ScalarFunction
#[derive(Clone, Debug)]
pub struct ScalarFunctionLambdaArg {
    /// The parameters defined in this lambda
    ///
    /// For example, for `array_transform([2], v -> -v)`,
    /// this will be `vec![Field::new("v", DataType::Int32, true)]`
    pub params: Vec<FieldRef>,
    /// The body of the lambda
    ///
    /// For example, for `array_transform([2], v -> -v)`,
    /// this will be the physical expression of `-v`
    pub body: Arc<dyn PhysicalExpr>,
    /// A RecordBatch containing at least the captured columns inside this lambda body, if any
    /// Note that it may contain additional, unspecified columns, but that's an implementation detail
    ///
    /// For example, for `array_transform([2], v -> v + a + b)`,
    /// this will be a `RecordBatch` with at least columns `a` and `b`
    pub captures: Option<RecordBatch>,
}

impl ScalarFunctionArgs {
    pub fn to_lambda_args(&self) -> Vec<ValueOrLambda<'_>> {
        match &self.lambdas {
            Some(lambdas) => std::iter::zip(&self.args, lambdas)
                .map(|(arg, lambda)| match lambda {
                    Some(lambda) => ValueOrLambda::Lambda(lambda),
                    None => ValueOrLambda::Value(arg),
                })
                .collect(),
            None => self.args.iter().map(ValueOrLambda::Value).collect(),
        }
    }
}

/// An argument to a higher-order scalar function
pub enum ValueOrLambda<'a> {
    Value(&'a ColumnarValue),
    Lambda(&'a ScalarFunctionLambdaArg),
}
ArrayTransform invoke_with_args implementation
impl ScalarUDFImpl for ArrayTransform {
    fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result<ColumnarValue> {
        // args.lambda_args allows the convenient match below, instead of inspecting both args.args and args.lambdas
        let lambda_args = args.to_lambda_args();
        let [list_value, lambda] = take_function_args(self.name(), &lambda_args)?;

        let (ValueOrLambda::Value(list_value), ValueOrLambda::Lambda(lambda)) =
            (list_value, lambda)
        else {
            return exec_err!(
                "{} expects a value followed by a lambda, got {:?}",
                self.name(),
                &lambda_args
            );
        };

        let list_array = list_value.to_array(args.number_rows)?;

        // if any column got captured, we need to adjust it to the values arrays,
        // duplicating values of list with multiple values and removing values of empty lists
        // list_indices is not cheap so is important to avoid it when no column is captured
        let adjusted_captures = lambda
            .captures
            .as_ref()
            .map(|captures| take_record_batch(captures, &list_indices(&list_array)?))
            .transpose()?;

        // use closures and merge_captures_with_lazy_args so that it calls only the needed ones based on the number of arguments
        // avoiding unnecessary computations
        let values_param = || Ok(Arc::clone(list_values(&list_array)?));
        let indices_param = || elements_indices(&list_array);

        // the order of the merged schema is an unspecified implementation detail that may change in the future,
        // using this function is the correct way to merge as it return the correct ordering and will change in sync
        // the implementation without the need for fixes. It also computes only the parameters requested
        let lambda_batch = merge_captures_with_lazy_args(
            adjusted_captures.as_ref(),
            &lambda.params, // ScalarUDF already merged the fields returned in lambdas_parameters with the parameters names defined in the lambda, so we don't need to
            &[&values_param, &indices_param],
        )?;

        // call the transforming expression with the record batch composed of the list values merged with captured columns
        let transformed_values = lambda
            .body
            .evaluate(&lambda_batch)?
            .into_array(lambda_batch.num_rows())?;

        let field = match args.return_field.data_type() {
            DataType::List(field)
            | DataType::LargeList(field)
            | DataType::FixedSizeList(field, _) => Arc::clone(field),
            _ => {
                return exec_err!(
                    "{} expected ScalarFunctionArgs.return_field to be a list, got {}",
                    self.name(),
                    args.return_field
                )
            }
        };

        let transformed_list = match list_array.data_type() {
            DataType::List(_) => {
                let list = list_array.as_list();

                Arc::new(ListArray::new(
                    field,
                    list.offsets().clone(),
                    transformed_values,
                    list.nulls().cloned(),
                )) as ArrayRef
            }
            DataType::LargeList(_) => {
                let large_list = list_array.as_list();

                Arc::new(LargeListArray::new(
                    field,
                    large_list.offsets().clone(),
                    transformed_values,
                    large_list.nulls().cloned(),
                ))
            }
            DataType::FixedSizeList(_, value_length) => {
                Arc::new(FixedSizeListArray::new(
                    field,
                    *value_length,
                    transformed_values,
                    list_array.as_fixed_size_list().nulls().cloned(),
                ))
            }
            other => exec_err!("expected list, got {other}")?,
        };

        Ok(ColumnarValue::Array(transformed_list))
    }
}

Changes in tree traversals

Using this query as an example:

create table t as select 1 as a, [[2, 3]] as b, 1 as c;

select a, b, c, array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) from t;

a | b | c | array_transform(b, (b, i) -> array_transform(b, b -> b + c + i))
1, [[2, 3]], 1, [[4, 5]]

Detailing the identifiers in the query:

                                                                                                
                                                                                                     
                                                                                                     
          definition of lambda first parameter, arbitrally named "b", the element of the array being 
          transformed. In the example, a List(Int32) column of [2, 3] . shadows t.b                  
            ^                                                                                        
            |                                                                                        
            | definition of lambda second   <--+                                                     
            | parameter arbitrally named "i"   |                                                     
            | the 1-based index of the element | definition of the only parameter of the lambda,     
            | being transformed: in the        | arbitrally named "b", the element of the array      
            | example, a Int32 column of "1"   | being transformed. Shadows the parameter "b"        
            |                                  | from outer lambda. The index parameter is omitted.   
            |                                  | In the example, a Int32 column with values "2, 3"   
            +-------------------------------+  |                        ^                            
                                            |  |                        |                            
                                            |  |                        |                            
        select a, b, c, array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) from t;     
               |  |  |                  |                            |       |   |   |               
               |  |  |                  v                            |       |   |   |               
               |  |  |  column "b" from table "t", t.b being passed  |       |   |   |               
               |  |  |  as argument to outer array_transform         |       |   |   |               
               |  |  v                                               |       |   |   |               
               |  | projection of column "c" from table "t", t.c     |       |   |   |               
               |  |                                                  |       |   |   |               
               |  v                                                  |       |   |   |               
               | projection of column "b" from table "t", t.b        |       |   |   |               
               |                                                     |       |   |   |               
               v                                                     |       |   |   |               
         projection of column "a" from table "t", t.a                |       |   |   |               
                                                                     |       |   |   |               
                                                                     v       |   |   |               
                         parameter "b" from outer lambda being passed        |   |   |               
                         as argument to the inner array_transform            |   |   |               
                                                                             v   |   |               
                                  reference to parameter "b" from inner lambda   |   |               
                                                                                 |   |               
                                                                                 v   |               
                               reference to column "c" captured from table "t", t.c  |               
                                                                                     |               
                                                                                     v               
                                    reference to parameter "i" captured from outer lambda            
                                                                                                

Note that:

1: lambdas may be nested
2: lambdas may capture columns from the outer scope, be it from the input plan or from another, outer lambdas.
3: lambdas introduces parameters that shadows columns from the outer scope
4: lambdas may support multiple parameters, and it is possible to omit the trailing ones that aren't used. Omitting unnecessary parameters positioned before an used parameter is currently not supported and may incur unnecessary computations

Representing columns referring lambda parameters while being able to differentiate them from regular columns in Expr tree traversals

Because they are identical to regular columns, it is intuitive to use the same logical and physical expression to represent columns referring to lambdas parameters. However, the existing tree traversals were made without taking into account that a column may refer to a lambda parameter, and not a column from the input plan, and so they would behave erratically. In the example query, projection pushdown would try to push the lambda parameter "i", which won't exist in table "t".

Another example:

fn minimize_join_filter(expr: Arc<dyn PhysicalExpr>, ...) -> JoinFilter {
    let mut used_columns = HashSet::new();
    expr.apply(|expr| {
        if let Some(col) = expr.as_any().downcast_ref::<Column>() {
            // if this is a lambda column, code below will break
            used_columns.insert(col.index());
        }
        Ok(TreeNodeRecursion::Continue)
    });
    ...
}

Therefore, we either make available a way to differentiate them, or use two different expressions:

Option 1. Use the same Column expression, differentiate with a new set of TreeNode methods, *_with_lambdas_params

This PR uses the existing column expression, always unqualified, and adds a new set of TreeNode-like methods on expressions that starts traversals with an empty HashSet, and while traversing the expr tree, when finding a lambda, clone the set and adds the lambda parameters to it, and pass it to the visiting/transforming closure so that it can differentiate the columns

impl Expr {
    pub fn transform_with_lambdas_params<
        F: FnMut(Self, &HashSet<String>) -> Result<Transformed<Self>>,
    >(
        self,
        mut f: F,
    ) -> Result<Transformed<Self>> {}
}
Expr tree traversal with lambdas_params. This query is a modified version of the example query where the inner lambda second parameter is used
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷ array_transform(b, (b, i) -> array_transform(b, (b, j) -> b + i + j)) ╷        
╷                                                                       ╷        
╷                                                                       ╷        
╷                        lambdas_params = {}                            ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                    │                                            
                                    ▼                                            
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷             (b, i) -> array_transform(b, (b, j) -> b + i + j)         ╷        
╷                                                                       ╷        
╷                                                                       ╷        
╷                     lambdas_params = { b, i }                         ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                    │                                            
                                    ▼                                            
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                 array_transform(b, (b, j) -> b + i + j)               ╷        
╷                                                                       ╷        
╷                                                                       ╷        
╷                     lambdas_params = { b, i }                         ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                    │                                            
                                    ▼                                            
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐       
╷                          (b, j) -> b + i + j                           ╷       
╷                                                                        ╷       
╷                                                                        ╷       
╷                      lambdas_params = { b, i, j }                      ╷       
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘       

How minimize_join_filter would looks like:

fn minimize_join_filter(expr: Arc<dyn PhysicalExpr>, ...) -> JoinFilter {
    let mut used_columns = HashSet::new();
    expr.apply_with_lambdas_params(|expr, lambdas_params| {
        if let Some(col) = expr.as_any().downcast_ref::<Column>() {
            // dont include lambdas parameters
            if !lambdas_params.contains(col.name()) {
                used_columns.insert(col.index());
            }
        }
        Ok(TreeNodeRecursion::Continue)
    })
    ...
}

I started with this option without an idea of how big it would be. It requires using the new TreeNode methods and checking the column in 30+ tree traversals, and so will downstream too. I choose to keep it in this PR so that we only choose/discard this knowing how big it is.

Option 2. New Expr LambdaColumn

Create a new Expr, LambdaColumn, which doesn't require new TreeNode methods, but requires that expr_api users use this new expr when applicable. It requires a fix in expr_simplifier for expressions with a lambda column, and may require similar work in downstream.

struct LambdaColumn {
    name: String,
    spans: Spans,
}

struct LambdaColumn {
    name: String,
    index: usize,
}

Existing code inalterated

fn minimize_join_filter(expr: Arc<dyn PhysicalExpr>, ...) -> JoinFilter {
    let mut used_columns = HashSet::new();
    expr.apply(|expr| {
        if let Some(col) = expr.as_any().downcast_ref::<Column>() {
                //no need to check because Column is never a LambdaColumn
                used_columns.insert(col.index());
        }
        Ok(TreeNodeRecursion::Continue)
    })
    ...
}

Option 3. Add is_lambda_parameter boolean field to Column

Add is_lambda_parameter to the existing column expression. It won't require new TreeNode methods, but still requires checking the field everywhere that new TreeNode methods are currently used in this PR

//logical
struct Column {
    pub relation: Option<TableReference>,
    pub name: String,
    pub spans: Spans,
    pub is_lambda_parameter: bool,
}

//physical
struct Column {
    name: String,
    index: usize,
    is_lambda_parameter: bool,
}

How minimize_join_filter would look like:

fn minimize_join_filter(expr: Arc<dyn PhysicalExpr>, ...) -> JoinFilter {
    let mut used_columns = HashSet::new();
    expr.apply(|expr| {
        if let Some(col) = expr.as_any().downcast_ref::<Column>() {
            // dont include lambdas parameters
            if !col.is_lambda_parameter {
                used_columns.insert(col.index());
            }
        }
        Ok(TreeNodeRecursion::Continue)
    })
    ...
}

Comparison between options:

Expr::Column/ColumnExpr Expr::LambdaColumn/LambdaColumnExpr is_lambda_parameter flag in Column
Internal code change New TreeNode methods, use them where applicable Add new expr. Use the correct expr while parsing sql(instrumentation already exists to skip normalization of lambdas parameters in sql parsing) Check the field where applicable. Set the field while parsing sql
Downstream code change If lambda support is desired, use the new TreeNode where applicable. Otherwise none None If lambda support is desired, check the field where applicable Otherwise none
Create a new Expr No Yes No
Requires new TreeNode methods Yes, _with_lambdas_params for both logical and physical expressions No No
Inhibits existing optimizations for exprs with columns of lambdas parameters No expr_simplifier, fixable. But may happen in downstream too No
expr_api users must reason about No Yes, use the correct expression type Yes, set the flag
Two almost identical Expressions which major difference is the place in the expr tree that they exist No Yes No
Data from the tree leaks into the column node No No Yes, is_lambda_parameter is a information not about the column node itself, but about its place in the tree

Ultimately, I don't have an inclination for any option and I believe it's a decision up to maintainers and those with more contact with downstream users, who have more idea of what option is easier to use. I think that the most laborious being already implemented puts us in a good position to make a choice, and would be easy to change to another option.

Continue in the comment below.

@github-actions github-actions bot added sql SQL Planner logical-expr Logical plan and expressions physical-expr Changes to the physical-expr crates optimizer Optimizer rules core Core DataFusion crate sqllogictest SQL Logic Tests (.slt) substrait Changes to the substrait crate catalog Related to the catalog crate common Related to common crate proto Related to proto crate functions Changes to functions implementation datasource Changes to the datasource crate ffi Changes to the ffi crate physical-plan Changes to the physical-plan crate spark labels Nov 25, 2025
@gstvg
Copy link
Contributor Author

gstvg commented Nov 25, 2025

Traversing Expr trees with a schema that include lambdas parameters

The parameters of a lambda aren't present in the schema of the plan they belong to. During tree traversals that use a schema to check expressions datatype, nullability and metadata, there must be a way to access a schema which includes those parameters.

Expr tree traversal with wrong schema
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷ array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) ╷        
╷                                                                  ╷        
╷                        a = Int32                                 ╷        
╷                        b = List(List(Int32))                     ╷        
╷                        c = Int32                                 ╷        
╷                                                                  ╷        
╷                                                                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷             (b, i) -> array_transform(b, b -> b + c + i)         ╷        
╷                                                                  ╷        
╷                         a = Int32                                ╷        
╷                         b = List(List(Int32))                    ╷        
╷                         c = Int32                                ╷        
╷                                                                  ╷        
╷              !! missing "i", incorrect "b" type !!               ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                 array_transform(b, b -> b + c + i)               ╷        
╷                                                                  ╷        
╷                        a = Int32                                 ╷        
╷                        b = List(List(Int32))                     ╷        
╷                        c = Int32                                 ╷        
╷                                                                  ╷        
╷              !! missing "i", incorrect "b" type !!               ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                          b -> b + c + i                          ╷        
╷                                                                  ╷        
╷                          a = Int32                               ╷        
╷                          b = List(List(Int32))                   ╷        
╷                          c = Int32                               ╷        
╷                                                                  ╷        
╷              !! missing "i", incorrect "b" type !!               ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        

Option 1. Per-lambda schema with a new set of TreeNode methods: *_with_schema

Once again, this PR adds another set of TreeNode-like methods on logical and physical expressions, that while traversing expression trees, when they find a ScalarUDF that contains a lambda on its arguments, uses ScalarUDF::lambdas_parameters to create a schema adjusted for each of its arguments, and pass it as an argument to the visiting/transforming function.

impl Expr {
pub fn transform_with_schema<
        F: FnMut(Self, &DFSchema) -> Result<Transformed<Self>>,
    >(
        self,
        schema: &DFSchema,
        f: F,
    ) -> Result<Transformed<Self>> {}
}

Example usage:

pub fn infer_placeholder_types(self, schema: &DFSchema) -> Result<(Expr, bool)> {
        let mut has_placeholder = false;
        // Provide the schema as the first argument. 
        // Transforming closure receive an adjusted_schema as argument
        self.transform_with_schema(schema, |mut expr, adjusted_schema| {
            match &mut expr {
                // Default to assuming the arguments are the same type
                Expr::BinaryExpr(BinaryExpr { left, op: _, right }) => {
                    // use adjusted_schema and not schema. Those expressions may contain 
                    // columns referring to a lambda parameter, which Field would only be
                    // available in adjusted_schema and not in schema
                    rewrite_placeholder(left.as_mut(), right.as_ref(), adjusted_schema)?;
                    rewrite_placeholder(right.as_mut(), left.as_ref(), adjusted_schema)?;
                }
    ....

In order to add the lambda parameters to schema, we need to take into account DFSchema properties:

"Unqualified fields must be unique not only amongst themselves, but also must have a distinct name from any qualified field names"

Since lambdas parameters are always unqualified, they may conflict with columns of the outer schema, even though those being qualified. To fix this conflict, we can either:

1: Replace the existing column with the lambda parameter, in the same index of the vec of fields of the schema, in order to not change the index of columns to the right of it. That's the current approach in this PR

Expr tree traversal with adjusted schema, replacing conflicts
 +------------------------------------------------------------------+  
 | array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) |  
 |                                                                  |  
 |                         a = Int32                                |  
 |                         b = List(List(Int32))                    |  
 |                         c = Int32                                |  
 +------------------------------------------------------------------+  
                                   |                                   
                                   |                                   
                                   v                                   
 +------------------------------------------------------------------+  
 |             (b, i) -> array_transform(b, b -> b + c + i)         |  
 |                                                                  |  
 |                         a = Int32                                |  
 |                         b = List(Int32)  ! replaced !            |  
 |                         c = Int32                                |  
 |                         i = Int32                                |  
 +------------------------------------------------------------------+  
                                   |                                   
                                   |                                   
                                   v                                   
 +------------------------------------------------------------------+  
 |                 array_transform(b, b -> b + c + i)               |  
 |                                                                  |  
 |                         a = Int32                                |  
 |                         b = List(Int32)   ! replaced !           |  
 |                         c = Int32                                |  
 |                         i = Int32                                |  
 +------------------------------------------------------------------+  
                                   |                                   
                                   |                                   
                                   v                                   
 +------------------------------------------------------------------+ 
 |                          b -> b + c + i                          | 
 |                                                                  | 
 |                           a = Int32                              | 
 |                           b = Int32     ! replaced !             | 
 |                           c = Int32                              | 
 |                           i = Int32                              | 
 +------------------------------------------------------------------+ 
                                                                      


2: Rename the shadowed column to an unique, non-conflicting name and add the lambda parameter to the end of the vec of fields of the schema. This option allows checking if a physical column refers to a lambda parameter by checking if its index is greater or equal than the number of fields of the outer schema. When this information is available, it eliminates the need to use the with_lambdas_params variations of TreeNode methods. It's trivial to change the PR to use this.

Expr tree traversal with adjusted schema, renaming conflicts
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷ array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                        a = Int32                                 ╷        
╷                        b = List(List(Int32))                     ╷        
╷                        c = Int32                                 ╷        
╷                                                                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷             (b, i) -> array_transform(b, b -> b + c + i)         ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                         a = Int32                                ╷        
╷                         b_shadowed1 = List(List(Int32))          ╷        
╷                         c = Int32                                ╷        
╷                         b = List(Int32)                          ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                 array_transform(b, b -> b + c + i)               ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                        a = Int32                                 ╷        
╷                        b_shadowed1 = List(List(Int32))           ╷        
╷                        c = Int32                                 ╷        
╷                        b = List(Int32)                           ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                          b -> b + c + i                          ╷        
╷                                                                  ╷        
╷                          a = Int32                               ╷        
╷                          b_shadowed1 = List(List(Int32))         ╷        
╷                          c = Int32                               ╷        
╷                          b_shadowed2 = List(Int32)               ╷        
╷                          b = Int32                               ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        



Lambdas usually are evaluated with a different number of rows than that of the outer scope, as in the example, where array_transform is executed with one row, and its lambda with two rows, one for each element of the array. The UDF implementation is responsible for adjusting the captured columns with the number of rows of its parameters with whatever logic makes sense to it. For array_transform, its to copy the value of the captured column for each element of the arrays:
        copied once  a [1]------------------> a 1  
                                                   
     copied 2 times  b [2, 3] --------------> b 2  
                               \                   
         not copied  c []       ------------> b 3     
                                                

This adjustment is costly, so it's necessary to provide a way to the implementation to avoid adjusting uncaptured columns.

It's intuitive to just remove the uncaptured columns, but note in the diagram and in the query below that it can change the index of captured columns. The "c" column has index 2 in the outer scope but ends up with index 1 in the others scopes

Expr tree traversal with a schema with uncaptured columns removed
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷ array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                        a@0 = Int32                               ╷        
╷                        b@1 = List(List(Int32))                   ╷        
╷                        c@2 = Int32                               ╷        
╷                                                                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷             (b, i) -> array_transform(b, b -> b + c + i)         ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                         b@0 = List(Int32)                        ╷        
╷                         c@1 = Int32                              ╷        
╷                         i@2 = Int32                              ╷        
╷                                                                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                 array_transform(b, b -> b + c + i)               ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                         b@0 = List(Int32)                        ╷        
╷                         c@1 = Int32                              ╷        
╷                         i@2 = Int32                              ╷        
╷                                                                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                          b -> b + c + i                          ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                          b@0 = Int32                             ╷        
╷                          c@1 = Int32                             ╷        
╷                          i@2 = Int32                             ╷        
╷                                                                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        

select a@0, b@1, c@2, array_transform(b@0, (b@0, i@2) -> array_transform(b@0, b@0 -> b@0 + c@1 + i@2)) from t;

Option 1a: Nullify uncaptured columns

To keep the indices stable, this PR won't remove uncaptured columns, as such, they are still present in the adjusted schema with their original type during tree traversals with the new _with_schema methods. However, to avoid the costly adjustment, when they are passed to the UDF in invoke_with_args, they are substituted with columns with the Null datatype.

Expr execution/evaluation RecordBatch schema with uncaptured columns substituted with Null columns
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷ array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                        a = Int32                                 ╷        
╷                        b = List(List(Int32))                     ╷        
╷                        c = Int32                                 ╷        
╷                                                                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷             (b, i) -> array_transform(b, b -> b + c + i)         ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                         a = Null  ! nullified !                  ╷        
╷                         b = List(Int32)                          ╷        
╷                         c = Int32                                ╷        
╷                         i = Int32                                ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                 array_transform(b, b -> b + c + i)               ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                        a = Null  ! nullified !                   ╷        
╷                        b = List(Int32)                           ╷        
╷                        c = Int32                                 ╷        
╷                        i = Int32                                 ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                          b -> b + c + i                          ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                          a = Null  ! nullified !                 ╷        
╷                          b = Int32                               ╷        
╷                          c = Int32                               ╷        
╷                          i = Int32                               ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        

Option 1b TreeNode *_with_indices_mapping

To avoid keeping uncaptured columns in the schema and substituting them with null in the batch, is possible to add another set of TreeNode-like methods on physical expressions that calls the visiting/transforming function with a second parameter of type HashMap<usize, usize> mapping the indices of the current scope with the ones from the outermost scope. This requires that before calling the visiting/transforming function for a physical lambda expression, all its subtree be visited to collect all the captured columns to build the indices mapping. Inner lambdas require the process again and can't reuse the work of the outer lambda. This may be costly for lambdas with complex expressions and/or highly nested.

impl PhysicalExprExt for Arc<dyn PhysicalExpr> {
    pub fn transform_with_indices_mapping<
        F: FnMut(Self, &HashMap<usize, usize>) -> Result<Transformed<Self>>,
    >(
        self,
        mut f: F,
    ) -> Result<Transformed<Self>> {}
}
Expr tree traversal with indices_mapping: "c" has index 2 in the root scope but 1 in the others
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷ array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                        indices_mapping = {}                      ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷             (b, i) -> array_transform(b, b -> b + c + i)         ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                     indices_mapping = { 1 => 2 }                 ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                 array_transform(b, b -> b + c + i)               ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                    indices_mapping = { 1 => 2 }                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                          b -> b + c + i                          ╷        
╷                                                                  ╷        
╷                                                                  ╷        
╷                    indices_mapping = { 1 => 2 }                  ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        

The code on minimize_join_filter would change to:

fn minimize_join_filter(expr: Arc<dyn PhysicalExpr>, ...) -> JoinFilter {
    let mut used_columns = HashSet::new();
    expr.apply_with_indices_mapping(|expr, indices_mapping| {
        if let Some(col) = expr.as_any().downcast_ref::<Column>() {
            // this column may be child of a lambda, where this indice would refer to the lambda
            // scoped schema, which won't include uncaptured columns from the plan input,
            // and therefore may differ from the indices of the schema of the input plan
            // In such cases, indices_mapping contain the mapping to the index of the input plan
            // if a mapping is not found, it should be a column referring to a lambda parameter
            let scoped_index = col.index();
            if let Some(plan_index) = indices_mapping.get(scoped_index) {
                used_columns.insert(plan_index);
            }
        }
        Ok(TreeNodeRecursion::Continue)
    })
    ...
}

Option 2. Create a schema with all parameters from all lambdas for tree traversals

Use a secondary schema containing all parameters from all lambdas. For that, expressions must be transformed, normalizing all lambda parameters and its references, with a unique qualifier per lambda, so they can coexist without conflicts in this schema. A qualifier field would be added to the lambda expr

pub struct Lambda {
    pub qualifier: Option<String>,
    pub params: Vec<String>,
    pub body: Box<Expr>,
}

Schema of the example:

t.a: Int32
t.b: List(List(Int32))
lambda1.b: List(Int32)
lambda1.i: UInt32
lambda2.b: Int32

From my understanding of this video, this is similar to what DuckDB does on its binder, although with differences in the evaluation part. I didn't find any other resource for other open source engines with lambda support, like Clickhouse and Spark.

This works well when dealing with plans nodes, where, during plan creation time or schema recomputation, we can normalize its lambdas, create the extended schema and save it as plan field, exposing it with a method like "lambda_extended_schema", although with an added cost to plan creation/schema recomputation. The lambda normalization actually requires two passes, a first to collect any existing lambda qualifier to avoid reusing them in the last, normalizing pass.

How code would look like:

//from
expr.transform_with_schema(plan.schema(), |node, adjusted_schema| ...)
//to
let schema = plan.lambda_extended_schema();
expr.transform(|node| ...)

Another example:

impl LogicalPlan {
    pub fn replace_params_with_values(
            self,
            param_values: &ParamValues,
        ) -> Result<LogicalPlan> {
            self.transform_up_with_subqueries(|plan| {
                // use plan.lambda_extended_schema() containing lambdas parameters
                // instead of plan.schema() which wont
                let lambda_extended_schema = Arc::clone(plan.lambda_extended_schema());
                let name_preserver = NamePreserver::new(&plan);
                plan.map_expressions(|e| {
                    // if this expression is child of lambda and contain columns referring it's parameters
                    // the lambda_extended_schema already contain them
                    let (e, has_placeholder) = e.infer_placeholder_types(&lambda_extended_schema)?;
    ....

However, when working with functions/methods that deal directly with expressions, detached from a plan, the expression lambdas may be unnormalized, and the extended schema is unavailable. There's a few public methods/functions like that, like infer_placeholder_types for example:

impl Expr {
    pub fn infer_placeholder_types(self, schema: &DFSchema) -> Result<(Expr, bool)> {
        let mut has_placeholder = false;
        self.transform(|mut expr| ...)
        ...
    }
}   

It could either:

1: Require to be only called with normalized expressions, and that the schema argument be the extended schema, or return an error otherwise, which is restrictive and put strain on users

2: Allow to be called with unnormalized expressions, visit the whole expr tree collecting the existing lambdas qualifiers to avoid to avoid duplicate qualifiers in the next step, perform a first transformation to guarantee that the expression lambdas are normalized, create the extended schema, for only then perform the second transformation to infer the placeholder types using the extended schema. While it can document that the returned expression is normalized, it's still a regular Expr which doesn't encode that property in its type. Also, without changing the method signature, it wouldn't even be possible to return the extended schema to allow it to be used again in other places without recomputation. This is costly and won't allow reuse of its costly work



Normalized example:

select t.a, t.b, array_transform(t.b, (lambda1.b, lambda1.i) -> array_transform(lambda1.b, lambda2.b -> lambda2.b + t.a + lambda1.i)) from t;

Just like with the first option, this also sets uncaptured columns to Null, as well as unavailable/out-of-scope lambdas parameters.

Expr tree batch evaluation with a single extended schema
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷ array_transform(b, (b, i) -> array_transform(b, b -> b + c + i)) ╷        
╷                                                                  ╷        
╷                        t.a = Int32                               ╷        
╷                        t.b = List(List(Int32))                   ╷        
╷                        t.c = Int32                               ╷        
╷                        lambda1.b = Null                          ╷        
╷                        lambda1.i = Null                          ╷        
╷                        lambda2.b = Null                          ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷             (b, i) -> array_transform(b, b -> b + c + i)         ╷        
╷                                                                  ╷        
╷                        t.a = Null                                ╷        
╷                        t.b = Null                                ╷        
╷                        t.c = Int32                               ╷        
╷                        lambda1.b = List(Int32)                   ╷        
╷                        lambda1.i = Int32                         ╷        
╷                        lambda2.b = Null                          ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                 array_transform(b, b -> b + c + i)               ╷        
╷                                                                  ╷        
╷                        t.a = Null                                ╷        
╷                        t.b = Null                                ╷        
╷                        t.c = Int32                               ╷        
╷                        lambda1.b = List(Int32)                   ╷        
╷                        lambda1.i = Int32                         ╷        
╷                        lambda2.b = Null                          ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        
                                 │                                          
                                 ▼                                          
┌╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┐        
╷                          b -> b + c + i                          ╷        
╷                                                                  ╷        
╷                          t.a = Null                              ╷        
╷                          t.b = Null                              ╷        
╷                          t.c = Int32                             ╷        
╷                          lambda1.b = Null                        ╷        
╷                          lambda1.i = Int32                       ╷        
╷                          lambda2.b = Int32                       ╷        
└╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶╶┘        

With this option, indices are always stable across the whole tree

This option allows checking if a physical column refers to a lambda parameter by checking if its index is greater or equal than the number of fields of the outer schema. When this information is available, it eliminates the need to use the with_lambdas_params variations of TreeNode methods.

Comparison between options:

* per-lambda schema with uncaptured columns set to null per-lambda schema with indices_mapping single extended schema
New set of TreeNode methods Yes, 1, _with_schema for both logical and physical expressions Yes, 2, _with_schema for both logical and physical expressions, and _with_indices_mapping for physical expressions No
Tree traversal added cost Only when encountering a lambda Only when encountering a lambda Zero
Plan creation/ recompute schema added cost Zero Zero Always, regardless of existence of any lambda
Code change, internal New set of TreeNode methods and using them instead of the current ones when applicable 2 new set of TreeNode methods and using them instead of the current ones when applicable Untried, unpredictable
Code change, downstream if lambda support is desired, use the new TreeNode methods instead of the current ones when applicable otherwise none if lambda support is desired, use the new TreeNode methods instead of the current ones when applicable otherwise none Variable, medium when closely associated with a Plan, just call plan. lambda_extended_schema() Unpredictable when plan is unavailable or doesn't exist
Change uncaptured columns DataType to Null Yes No Yes
Presence of unneeded Null columns in the schema during planning and optimizing and in the RecordBatch during execution as a padding/filler to keep indices stable Yes No Yes
Stable column indices across the whole expr tree Yes No Yes
Make _with_lambdas_params unnecessary for physical expressions if Expr::Column is used No Yes No

Splitting this into smaller PRs

If this PR is decided to move forward, it will likely be with smaller PRs. In that case, I already planned a division shown below. It's necessary to analyze the graph, as it doesn't help with the discussion of this text, and it's included here just to show a reasonable granularity of smaller PRs I could find in case it helps decide whether to move this forward or not.

Each rectangular node is a PR. Asymmetrical nodes are a collection of smaller PRs which share the same dependencies and aren't a dependency of any other PR. Green ones can be opened immediately. Gray ones contain unmet dependencies. Merged PRs will be colored with blue.

Left-to-Right full-screen link

graph LR
    classDef blue fill:blue
    classDef green fill:green

    subgraph "SQL" [SQL 84LOC]
        SQLP[SQL Parse 65LOC]
        SQLU[SQL Unparse 19LOC]
    end
    
    LL[Logical Expr::Lambda 100LOC]:::green
    LL --> CSE[CSE 50LOC]
    P[Plan logical lambda into physical expr 25LOC]
    UDF1["Extend ScalarUDF[Impl] 300LOC"]
    UDF2[ScalarUDF lambda schema helpers 100LOC]
    AM[Non functional array_transform 270LOC]
    PSFL[ScalarFunction PhysicalExpr lambda aware 30LOC]

    PL --> PSFL
    UDF1 --> PSFL
    UDF1 --> AM

    %%CILP[Column::is_lambda_parameter 6LOC]
    
    subgraph "Expr::*_with_schema"
        LTN[Expr::*_with_schema API def 50LOC]:::green
        LTNB[make Expr::*_with_schema lambda aware 70LOC]
        LTN --> LTNES[Expr Simplifier 100LOC]
        LTNA>"
            use Expr::*_with_schema in existing code
            Type Coercion 110LOC
            normalize_col[with_schemas_and_ambiguity_check] 31LOC
            SessionState::create_physical_expr 4LOC
            Expr::infer_placeholder_type 1LOC
            ApplyFunctionRewrites 10LOC
            optmize_projections::rewrite_expr 2LOC
            SqlToRel::try_process_group_by_unnest 10LOC
            sql resolve_columns 5LOC
            Example type_coercion_demo 10LOC
        "]
    end
    
    PL[Physical Lambda Expr 108LOC]:::green
    
    subgraph "PhysicalExpr::*_with_schema"
        PTN[PhysicalExpr::*_with_schema API def 25LOC]:::green
        PTNA>"
            use PhysicalExpr::*_with_schema in ProjectionMapping 32LOC
            PhysicalExprSimplifier 20LOC
            unwrap_cast_in_comparison 10LOC
            AsyncMapper::find_references 1LOC
            Example CustomCastsPhysicalExprAdapter 10LOC
        "]
        PTNB[make PhysicalExpr::*_with_schema lambda aware 160LOC]
    end
    
    subgraph capture
        CS[Capture support 30LOC]
        LCH["List capture helpers 60LOC(4x15)"]
        MCH["Merge capture with arguments helper 66LOC(2x33)"]
        AMCT[array_transform capture support 20LOC]
    end

    PSFL --> CS
    PTN --> CS
    LL --> SQL
    LL --> P
    UDF2 --> LTNES
    UDF2 --> LTNB
    UDF2 --> PTNB
    LTN --> LTNA
    LTN --> LTNB
    LL --> LTNB
    LTN --> UDF2
    LL --> UDF1 --> UDF2
    UDF2 --> P
    PL --> P
    PL --> PTNB
    PTN --> PTNB
    PTN --> PTNA
    AM --> AMCT
    CS --> AMCT
    LCH --> AMCT
    MCH --> AMCT

    LL --> LTNLP2

subgraph "Expr::*_with_lambdas_params"
        LTNLP --> LTNLP2[make Expr::*_with_lambdas_params lambda aware]
        LTNLP[Expr::*_with_lambdas_params API def 50LOC]:::green -->
        AAAAA>"
            expr_applicable_for_cols 5LOC
            Expr::add_column_refs[_counts]/any_column_refs 15LOC
            expr_to_columns 10LOC
            columnize_expr 15LOC
            find_columns_referenced_by_expr 10LOC
            find_column_indexes_referenced_by_expr 5LOC
            normalize_col 10LOC
            normalize_col_with_schemas_and_ambiguity_check 15LOC
            replace_col 10LOC
            transform_up_with_lambdas_params 10LOC
            filter_exprs_evaluation_result_on_empty_batch 10LOC
            replace_cols_by_name 10LOC
            optimizer::push_down_filter::contain 15LOC
            ScalarSubqueryToJoin 40LOC
            TableAliasRewriter 20LOC
            Example ShreddedJsonRewriter 30LOC
        "]
    end

    PL --> PTNLP2

    subgraph "PhysicalExpr::*_with_lambdas_params"
        PTNLP --> PTNLP2[make PhysicalExpr::*_with_lambdas_params lambda aware]

        PTNLP[PhysicalExpr::*_with_lambdas_params API def 50LOC]:::green -->
        BBBBB>"
            CustomPhysicalExprAdapter 3LOC
            ParquetPushdownChecker 13LOC
            add_offset_to_expr 3LOC
            update_expr 10LOC
            project_ordering 20LOC
            with_new_schema 10LOC
            collect_columns 10LOC
            reassign_expr_columns 10LOC
            DefaultPhysicalExprAdapter 40LOC
            projection pushdown 50LOC
            sort pushdown 40LOC
            projection 50LOC
            stream_join_utils 40LOC
            pruning predicate rewrite_column_expr 5LOC
            Example DefaultValuePhysicalExprAdapter 20LOC
        "]
    end


Loading


Top-to-Bottom full-screen link

graph TB
    classDef blue fill:blue
    classDef green fill:green

    subgraph "SQL" [SQL 84LOC]
        SQLP[SQL Parse 65LOC]
        SQLU[SQL Unparse 19LOC]
    end
    
    LL[Logical Expr::Lambda 100LOC]:::green
    LL --> CSE[CSE 50LOC]
    P[Plan logical lambda into physical expr 25LOC]
    UDF1["Extend ScalarUDF[Impl] 300LOC"]
    UDF2[ScalarUDF lambda schema helpers 100LOC]
    AM[Non functional array_transform 270LOC]
    PSFL[ScalarFunction PhysicalExpr lambda aware 30LOC]

    PL --> PSFL
    UDF1 --> PSFL
    UDF1 --> AM

    %%CILP[Column::is_lambda_parameter 6LOC]
    
    subgraph "Expr::*_with_schema"
        LTN[Expr::*_with_schema API def 50LOC]:::green
        LTNB[make Expr::*_with_schema lambda aware 70LOC]
        LTN --> LTNES[Expr Simplifier 100LOC]
        LTNA>"
            use Expr::*_with_schema in existing code
            Type Coercion 110LOC
            normalize_col[with_schemas_and_ambiguity_check] 31LOC
            SessionState::create_physical_expr 4LOC
            Expr::infer_placeholder_type 1LOC
            ApplyFunctionRewrites 10LOC
            optmize_projections::rewrite_expr 2LOC
            SqlToRel::try_process_group_by_unnest 10LOC
            sql resolve_columns 5LOC
            Example type_coercion_demo 10LOC
        "]
    end
    
    PL[Physical Lambda Expr 108LOC]:::green
    
    subgraph "PhysicalExpr::*_with_schema"
        PTN[PhysicalExpr::*_with_schema API def 25LOC]:::green
        PTNA>"
            use PhysicalExpr::*_with_schema in ProjectionMapping 32LOC
            PhysicalExprSimplifier 20LOC
            unwrap_cast_in_comparison 10LOC
            AsyncMapper::find_references 1LOC
            Example CustomCastsPhysicalExprAdapter 10LOC
        "]
        PTNB[make PhysicalExpr::*_with_schema lambda aware 160LOC]
    end
    
    subgraph capture
        CS[Capture support 30LOC]
        LCH["List capture helpers 60LOC(4x15)"]
        MCH["Merge capture with arguments helper 66LOC(2x33)"]
        AMCT[array_transform capture support 20LOC]
    end

    PSFL --> CS
    PTN --> CS
    LL --> SQL
    LL --> P
    UDF2 --> LTNES
    UDF2 --> LTNB
    UDF2 --> PTNB
    LTN --> LTNA
    LTN --> LTNB
    LL --> LTNB
    LTN --> UDF2
    LL --> UDF1 --> UDF2
    UDF2 --> P
    PL --> P
    PL --> PTNB
    PTN --> PTNB
    PTN --> PTNA
    AM --> AMCT
    CS --> AMCT
    LCH --> AMCT
    MCH --> AMCT

    LL --> LTNLP2

subgraph "Expr::*_with_lambdas_params"
        LTNLP --> LTNLP2[make Expr::*_with_lambdas_params lambda aware]
        LTNLP[Expr::*_with_lambdas_params API def 50LOC]:::green -->
        AAAAA>"
            expr_applicable_for_cols 5LOC
            Expr::add_column_refs[_counts]/any_column_refs 15LOC
            expr_to_columns 10LOC
            columnize_expr 15LOC
            find_columns_referenced_by_expr 10LOC
            find_column_indexes_referenced_by_expr 5LOC
            normalize_col 10LOC
            normalize_col_with_schemas_and_ambiguity_check 15LOC
            replace_col 10LOC
            transform_up_with_lambdas_params 10LOC
            filter_exprs_evaluation_result_on_empty_batch 10LOC
            replace_cols_by_name 10LOC
            optimizer::push_down_filter::contain 15LOC
            ScalarSubqueryToJoin 40LOC
            TableAliasRewriter 20LOC
            Example ShreddedJsonRewriter 30LOC
        "]
    end

    PL --> PTNLP2

    subgraph "PhysicalExpr::*_with_lambdas_params"
        PTNLP --> PTNLP2[make PhysicalExpr::*_with_lambdas_params lambda aware]

        PTNLP[PhysicalExpr::*_with_lambdas_params API def 50LOC]:::green -->
        BBBBB>"
            CustomPhysicalExprAdapter 3LOC
            ParquetPushdownChecker 13LOC
            add_offset_to_expr 3LOC
            update_expr 10LOC
            project_ordering 20LOC
            with_new_schema 10LOC
            collect_columns 10LOC
            reassign_expr_columns 10LOC
            DefaultPhysicalExprAdapter 40LOC
            projection pushdown 50LOC
            sort pushdown 40LOC
            projection 50LOC
            stream_join_utils 40LOC
            pruning predicate rewrite_column_expr 5LOC
            Example DefaultValuePhysicalExprAdapter 20LOC
        "]
    end

Loading


Right-to-Left full-screen link

graph RL
    classDef blue fill:blue
    classDef green fill:green

    subgraph "SQL" [SQL 84LOC]
        SQLP[SQL Parse 65LOC]
        SQLU[SQL Unparse 19LOC]
    end
    
    LL[Logical Expr::Lambda 100LOC]:::green
    LL --> CSE[CSE 50LOC]
    P[Plan logical lambda into physical expr 25LOC]
    UDF1["Extend ScalarUDF[Impl] 300LOC"]
    UDF2[ScalarUDF lambda schema helpers 100LOC]
    AM[Non functional array_transform 270LOC]
    PSFL[ScalarFunction PhysicalExpr lambda aware 30LOC]

    PL --> PSFL
    UDF1 --> PSFL
    UDF1 --> AM

    %%CILP[Column::is_lambda_parameter 6LOC]
    
    subgraph "Expr::*_with_schema"
        LTN[Expr::*_with_schema API def 50LOC]:::green
        LTNB[make Expr::*_with_schema lambda aware 70LOC]
        LTN --> LTNES[Expr Simplifier 100LOC]
        LTNA>"
            use Expr::*_with_schema in existing code
            Type Coercion 110LOC
            normalize_col[with_schemas_and_ambiguity_check] 31LOC
            SessionState::create_physical_expr 4LOC
            Expr::infer_placeholder_type 1LOC
            ApplyFunctionRewrites 10LOC
            optmize_projections::rewrite_expr 2LOC
            SqlToRel::try_process_group_by_unnest 10LOC
            sql resolve_columns 5LOC
            Example type_coercion_demo 10LOC
        "]
    end
    
    PL[Physical Lambda Expr 108LOC]:::green
    
    subgraph "PhysicalExpr::*_with_schema"
        PTN[PhysicalExpr::*_with_schema API def 25LOC]:::green
        PTNA>"
            use PhysicalExpr::*_with_schema in ProjectionMapping 32LOC
            PhysicalExprSimplifier 20LOC
            unwrap_cast_in_comparison 10LOC
            AsyncMapper::find_references 1LOC
            Example CustomCastsPhysicalExprAdapter 10LOC
        "]
        PTNB[make PhysicalExpr::*_with_schema lambda aware 160LOC]
    end
    
    subgraph capture
        CS[Capture support 30LOC]
        LCH["List capture helpers 60LOC(4x15)"]
        MCH["Merge capture with arguments helper 66LOC(2x33)"]
        AMCT[array_transform capture support 20LOC]
    end

    PSFL --> CS
    PTN --> CS
    LL --> SQL
    LL --> P
    UDF2 --> LTNES
    UDF2 --> LTNB
    UDF2 --> PTNB
    LTN --> LTNA
    LTN --> LTNB
    LL --> LTNB
    LTN --> UDF2
    LL --> UDF1 --> UDF2
    UDF2 --> P
    PL --> P
    PL --> PTNB
    PTN --> PTNB
    PTN --> PTNA
    AM --> AMCT
    CS --> AMCT
    LCH --> AMCT
    MCH --> AMCT

    LL --> LTNLP2

subgraph "Expr::*_with_lambdas_params"
        LTNLP --> LTNLP2[make Expr::*_with_lambdas_params lambda aware]
        LTNLP[Expr::*_with_lambdas_params API def 50LOC]:::green -->
        AAAAA>"
            expr_applicable_for_cols 5LOC
            Expr::add_column_refs[_counts]/any_column_refs 15LOC
            expr_to_columns 10LOC
            columnize_expr 15LOC
            find_columns_referenced_by_expr 10LOC
            find_column_indexes_referenced_by_expr 5LOC
            normalize_col 10LOC
            normalize_col_with_schemas_and_ambiguity_check 15LOC
            replace_col 10LOC
            transform_up_with_lambdas_params 10LOC
            filter_exprs_evaluation_result_on_empty_batch 10LOC
            replace_cols_by_name 10LOC
            optimizer::push_down_filter::contain 15LOC
            ScalarSubqueryToJoin 40LOC
            TableAliasRewriter 20LOC
            Example ShreddedJsonRewriter 30LOC
        "]
    end

    PL --> PTNLP2

    subgraph "PhysicalExpr::*_with_lambdas_params"
        PTNLP --> PTNLP2[make PhysicalExpr::*_with_lambdas_params lambda aware]

        PTNLP[PhysicalExpr::*_with_lambdas_params API def 50LOC]:::green -->
        BBBBB>"
            CustomPhysicalExprAdapter 3LOC
            ParquetPushdownChecker 13LOC
            add_offset_to_expr 3LOC
            update_expr 10LOC
            project_ordering 20LOC
            with_new_schema 10LOC
            collect_columns 10LOC
            reassign_expr_columns 10LOC
            DefaultPhysicalExprAdapter 40LOC
            projection pushdown 50LOC
            sort pushdown 40LOC
            projection 50LOC
            stream_join_utils 40LOC
            pruning predicate rewrite_column_expr 5LOC
            Example DefaultValuePhysicalExprAdapter 20LOC
        "]
    end

Loading


Bottom-to-Top full-screen link

graph BT
    classDef blue fill:blue
    classDef green fill:green

    subgraph "SQL" [SQL 84LOC]
        SQLP[SQL Parse 65LOC]
        SQLU[SQL Unparse 19LOC]
    end
    
    LL[Logical Expr::Lambda 100LOC]:::green
    LL --> CSE[CSE 50LOC]
    P[Plan logical lambda into physical expr 25LOC]
    UDF1["Extend ScalarUDF[Impl] 300LOC"]
    UDF2[ScalarUDF lambda schema helpers 100LOC]
    AM[Non functional array_transform 270LOC]
    PSFL[ScalarFunction PhysicalExpr lambda aware 30LOC]

    PL --> PSFL
    UDF1 --> PSFL
    UDF1 --> AM

    %%CILP[Column::is_lambda_parameter 6LOC]
    
    subgraph "Expr::*_with_schema"
        LTN[Expr::*_with_schema API def 50LOC]:::green
        LTNB[make Expr::*_with_schema lambda aware 70LOC]
        LTN --> LTNES[Expr Simplifier 100LOC]
        LTNA>"
            use Expr::*_with_schema in existing code
            Type Coercion 110LOC
            normalize_col[with_schemas_and_ambiguity_check] 31LOC
            SessionState::create_physical_expr 4LOC
            Expr::infer_placeholder_type 1LOC
            ApplyFunctionRewrites 10LOC
            optmize_projections::rewrite_expr 2LOC
            SqlToRel::try_process_group_by_unnest 10LOC
            sql resolve_columns 5LOC
            Example type_coercion_demo 10LOC
        "]
    end
    
    PL[Physical Lambda Expr 108LOC]:::green
    
    subgraph "PhysicalExpr::*_with_schema"
        PTN[PhysicalExpr::*_with_schema API def 25LOC]:::green
        PTNA>"
            use PhysicalExpr::*_with_schema in ProjectionMapping 32LOC
            PhysicalExprSimplifier 20LOC
            unwrap_cast_in_comparison 10LOC
            AsyncMapper::find_references 1LOC
            Example CustomCastsPhysicalExprAdapter 10LOC
        "]
        PTNB[make PhysicalExpr::*_with_schema lambda aware 160LOC]
    end
    
    subgraph capture
        CS[Capture support 30LOC]
        LCH["List capture helpers 60LOC(4x15)"]
        MCH["Merge capture with arguments helper 66LOC(2x33)"]
        AMCT[array_transform capture support 20LOC]
    end

    PSFL --> CS
    PTN --> CS
    LL --> SQL
    LL --> P
    UDF2 --> LTNES
    UDF2 --> LTNB
    UDF2 --> PTNB
    LTN --> LTNA
    LTN --> LTNB
    LL --> LTNB
    LTN --> UDF2
    LL --> UDF1 --> UDF2
    UDF2 --> P
    PL --> P
    PL --> PTNB
    PTN --> PTNB
    PTN --> PTNA
    AM --> AMCT
    CS --> AMCT
    LCH --> AMCT
    MCH --> AMCT

    LL --> LTNLP2

subgraph "Expr::*_with_lambdas_params"
        LTNLP --> LTNLP2[make Expr::*_with_lambdas_params lambda aware]
        LTNLP[Expr::*_with_lambdas_params API def 50LOC]:::green -->
        AAAAA>"
            expr_applicable_for_cols 5LOC
            Expr::add_column_refs[_counts]/any_column_refs 15LOC
            expr_to_columns 10LOC
            columnize_expr 15LOC
            find_columns_referenced_by_expr 10LOC
            find_column_indexes_referenced_by_expr 5LOC
            normalize_col 10LOC
            normalize_col_with_schemas_and_ambiguity_check 15LOC
            replace_col 10LOC
            transform_up_with_lambdas_params 10LOC
            filter_exprs_evaluation_result_on_empty_batch 10LOC
            replace_cols_by_name 10LOC
            optimizer::push_down_filter::contain 15LOC
            ScalarSubqueryToJoin 40LOC
            TableAliasRewriter 20LOC
            Example ShreddedJsonRewriter 30LOC
        "]
    end

    PL --> PTNLP2

    subgraph "PhysicalExpr::*_with_lambdas_params"
        PTNLP --> PTNLP2[make PhysicalExpr::*_with_lambdas_params lambda aware]

        PTNLP[PhysicalExpr::*_with_lambdas_params API def 50LOC]:::green -->
        BBBBB>"
            CustomPhysicalExprAdapter 3LOC
            ParquetPushdownChecker 13LOC
            add_offset_to_expr 3LOC
            update_expr 10LOC
            project_ordering 20LOC
            with_new_schema 10LOC
            collect_columns 10LOC
            reassign_expr_columns 10LOC
            DefaultPhysicalExprAdapter 40LOC
            projection pushdown 50LOC
            sort pushdown 40LOC
            projection 50LOC
            stream_join_utils 40LOC
            pruning predicate rewrite_column_expr 5LOC
            Example DefaultValuePhysicalExprAdapter 20LOC
        "]
    end

Loading

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

catalog Related to the catalog crate common Related to common crate core Core DataFusion crate datasource Changes to the datasource crate ffi Changes to the ffi crate functions Changes to functions implementation logical-expr Logical plan and expressions optimizer Optimizer rules physical-expr Changes to the physical-expr crates physical-plan Changes to the physical-plan crate proto Related to proto crate spark sql SQL Planner sqllogictest SQL Logic Tests (.slt) substrait Changes to the substrait crate

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Add support for lambda/higher order functions

1 participant