Skip to content

feat: Add datafusion-spark crate #15168

New issue

Have a question about this project? # for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “#”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? # to your account

Merged
merged 29 commits into from
May 1, 2025
Merged

feat: Add datafusion-spark crate #15168

merged 29 commits into from
May 1, 2025

Conversation

shehabgamin
Copy link
Contributor

@shehabgamin shehabgamin commented Mar 11, 2025

Which issue does this PR close?

Rationale for this change

See discussion in #5600

TL;DR Many projects want Spark-compatible expressions for use with DataFusion. There are some in Comet and there are some in the Sail project.

What changes are included in this PR?

Adding Spark crate.

Are these changes tested?

Are there any user-facing changes?

@github-actions github-actions bot added the functions Changes to functions implementation label Mar 11, 2025
@shehabgamin shehabgamin marked this pull request as ready for review March 13, 2025 02:15
@github-actions github-actions bot added core Core DataFusion crate sqllogictest SQL Logic Tests (.slt) labels Mar 13, 2025
specific language governing permissions and limitations
under the License.
-->

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Testing instructions here

# datafusion-spark: Spark-compatible Expressions

This crate provides Apache Spark-compatible expressions for use with DataFusion.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Testing instructions here

# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Example test here

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The expm1 probably works identically across all Spark versions and is not affected by different configuration settings, but many expressions are affected by settings such as ANSI mode and different date/time formats and timezones.

How do you think we should handle these different cases with this test approach?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A related question - There will be some functions that we can support with 100% compatibility and some that we cannot. It would be good to think about how we express that.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for all the questions, but I am really excited about this 😄 ... what would be involved in being able to run these same sqllogic test files in Spark (either in CI or manually locally) to confirm same/similar results

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

These are great questions and really good things to discuss. I'm about to go into a meeting but I have a bunch of thoughts that I'll share afterwards in a couple of hours.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The expm1 probably works identically across all Spark versions and is not affected by different configuration settings, but many expressions are affected by settings such as ANSI mode and different date/time formats and timezones.

How do you think we should handle these different cases with this test approach?

In the Sail code base, auxiliary information is passed into new() and stored within the struct. For example:
https://github.com/lakehq/sail/blob/be54362ce8bd79bfb3e55214c5a1b80c2c3d2492/crates/sail-plan/src/extension/function/datetime/timestamp_now.rs#L9-L32

#[derive(Debug)]
pub struct TimestampNow {
    signature: Signature,
    timezone: Arc<str>,
    time_unit: TimeUnit,
}

impl TimestampNow {
    pub fn new(timezone: Arc<str>, time_unit: TimeUnit) -> Self {
        Self {
            signature: Signature::nullary(Volatility::Stable),
            timezone,
            time_unit,
        }
    }

    pub fn timezone(&self) -> &str {
        &self.timezone
    }

    pub fn time_unit(&self) -> &TimeUnit {
        &self.time_unit
    }
}

And then in our PhysicalExtensionCodec we can do the following:
https://github.com/lakehq/sail/blob/be54362ce8bd79bfb3e55214c5a1b80c2c3d2492/crates/sail-execution/src/codec.rs#L946-L953

if let Some(func) = node.inner().as_any().downcast_ref::<TimestampNow>() {
            let timezone = func.timezone().to_string();
            let time_unit: gen_datafusion_common::TimeUnit = func.time_unit().into();
            let time_unit = time_unit.as_str_name().to_string();
            UdfKind::TimestampNow(gen::TimestampNowUdf {
                timezone,
                time_unit,
            })

If we decide to not use sqllogictest (per #15168 (comment)) then we will have no problem testing UDFs with auxiliary information. There are already tests in DataFusion core for this type of pattern as well:

async fn test_parameterized_scalar_udf() -> Result<()> {
let batch = RecordBatch::try_from_iter([(
"text",
Arc::new(StringArray::from(vec!["foo", "bar", "foobar", "barfoo"])) as ArrayRef,
)])?;
let ctx = SessionContext::new();
ctx.register_batch("t", batch)?;
let t = ctx.table("t").await?;
let foo_udf = ScalarUDF::from(MyRegexUdf::new("fo{2}"));
let bar_udf = ScalarUDF::from(MyRegexUdf::new("[Bb]ar"));
let plan = LogicalPlanBuilder::from(t.into_optimized_plan()?)
.filter(
foo_udf
.call(vec![col("text")])
.and(bar_udf.call(vec![col("text")])),
)?
.filter(col("text").is_not_null())?
.build()?;
assert_eq!(
format!("{plan}"),
"Filter: t.text IS NOT NULL\n Filter: regex_udf(t.text) AND regex_udf(t.text)\n TableScan: t projection=[text]"
);
let actual = DataFrame::new(ctx.state(), plan).collect().await?;
let expected = [
"+--------+",
"| text |",
"+--------+",
"| foobar |",
"| barfoo |",
"+--------+",
];
assert_batches_eq!(expected, &actual);
ctx.deregister_table("t")?;
Ok(())
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

A related question - There will be some functions that we can support with 100% compatibility and some that we cannot. It would be good to think about how we express that.

Throw errors when possible, and provide documentation. Depending on how shaky the compatibility for the function is, we may want to avoid implementing it altogether.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

... what would be involved in being able to run these same sqllogic test files in Spark (either in CI or manually locally) to confirm same/similar results

One option:

  1. A Python script to automatically generate Spark SQL function test cases and their results using PySpark.
  2. A README for developers explaining how to run the script and commit the test cases.
  3. An optional CI workflow to verify the correctness of the test cases' ground truth on demand.

Separate topic... Do you have ideas about fuzzy testing and its suitability in DataFusion?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is my suggestion.

  1. Write a Python script that generates interesting test data (probably in Parquet format) with edge cases using PySpark
  2. Create files containing SQL queries that operate on these test files
  3. Write a Python script to run those queries via PySpark and write results out to file
  4. Write a Rust script to run those queries using datafusion-spark and write results out to file
  5. Write a script that can compare the Spark and datafusion-spark output and report on any differences

Copy link
Contributor

@alamb alamb Mar 16, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The expm1 probably works identically across all Spark versions and is not affected by different configuration settings, but many expressions are affected by settings such as ANSI mode and different date/time formats and timezones.

I believe @Omega359 has a proposal here of how to thread the config options into the arguments of the functions (for the same reason)

# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Example test here

};
}

#[test]
Copy link
Contributor Author

@shehabgamin shehabgamin Mar 13, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Example invoke test here. Direct invocation tests should only be used to verify that the function is correctly implemented. Further tests are required in the sqllogictest crate (examples for ascii can be found in this PR).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My personal preference is to test them all from .slt rather than have any rust based tests unless there is something that can not be tested from .slt

For the different string types, we could perhaps cover the different string types using the same pattern as normal string tests -- see https://github.com/apache/datafusion/blob/main/datafusion/sqllogictest/test_files/string/README.md

However, I don't think this is required

};
}

#[test]
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Example invoke test here. Direct invocation tests should only be used to verify that the function is correctly implemented. Further tests are required in the sqllogictest crate (examples for expm1 can be found in this PR).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because sqllogictests are so much faster to write and update, I suggest we point people towards using sqllogictests to test the functions unless there is somehting that can not be tested using .slt files

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@andygrove and I found some correctness issues with sqllogictests. Specifically, we found issues with testing the correctness of floating point results.

The idea was to do something like this: #15168 (comment)

@alamb What are your thoughts on this? Should we perhaps use sqllogictest as long as we're not testing float point results and as long as the function being tested is not configurable?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the underlying sqlogictest library has the notion of "engines"

The one we use on main is here:
https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest/src/engines/datafusion_engine and

part of that is how to normalize the results into the strings printed in .slt tests

pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result<String> {
if !col.is_valid(row) {
// represent any null value with the string "NULL"
Ok(NULL_STR.to_string())
} else {
match col.data_type() {
DataType::Null => Ok(NULL_STR.to_string()),
DataType::Boolean => {
Ok(bool_to_str(get_row_value!(array::BooleanArray, col, row)))
}
DataType::Float16 => {
Ok(f16_to_str(get_row_value!(array::Float16Array, col, row)))
}
DataType::Float32 => {
Ok(f32_to_str(get_row_value!(array::Float32Array, col, row)))
}
DataType::Float64 => {
Ok(f64_to_str(get_row_value!(array::Float64Array, col, row)))
}
DataType::Decimal128(_, scale) => {
let value = get_row_value!(array::Decimal128Array, col, row);
Ok(decimal_128_to_str(value, *scale))
}
DataType::Decimal256(_, scale) => {
let value = get_row_value!(array::Decimal256Array, col, row);
Ok(decimal_256_to_str(value, *scale))
}
DataType::LargeUtf8 => Ok(varchar_to_str(get_row_value!(
array::LargeStringArray,
col,
row
))),
DataType::Utf8 => {
Ok(varchar_to_str(get_row_value!(array::StringArray, col, row)))
}
DataType::Utf8View => Ok(varchar_to_str(get_row_value!(
array::StringViewArray,
col,
row
))),
DataType::Dictionary(_, _) => {
let dict = col.as_any_dictionary();
let key = dict.normalized_keys()[row];
Ok(cell_to_string(dict.values(), key)?)
}
_ => {
let f =
ArrayFormatter::try_new(col.as_ref(), &DEFAULT_CLI_FORMAT_OPTIONS);
Ok(f.unwrap().value(row).to_string())
}
}
.map_err(DFSqlLogicTestError::Arrow)
}
}

If there is some different way to convert floating point values for spark maybe we could make a spark functions specific driver

The idea was to do something like this: #15168 (comment)

Ideally we could use one of the many existing tools in datafusion rather than write new scripts

Another potential possiblity might be to use insta.rs perhaps (which was added to the repo recently) which automates result update 🤔

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@alamb Thanks for the pointers, this seems reasonable to me.

@andygrove Does this sound good to you as well?

If we're all aligned, I think we've gathered enough input for me to push up some more code 🤠

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FYI I think @andygrove might be out for a week so he may not respond quickly

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@alamb Thanks for the pointers, this seems reasonable to me.

@andygrove Does this sound good to you as well?

If we're all aligned, I think we've gathered enough input for me to push up some more code 🤠

@andygrove @alamb Just checking in here

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for the late response here. I had a vacation and have been busy with Comet priorities since getting back. I would like to help with the review here. I do wonder if we could start with a smaller scope PR to get the initial crate in place.

I would also like to contribute some Spark-compatible shuffle implementation from Comet so that we can re-use it in Ballista.

}

fn name(&self) -> &str {
"spark_ascii"
Copy link
Contributor Author

@shehabgamin shehabgamin Mar 13, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Prefix with spark_ because sqllogictest evaluates both implementations of ascii.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see above

}

fn name(&self) -> &str {
"spark_expm1"
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Prefix with spark_ because sqllogictest may evaluate more than one implementation of expm1.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What I would recommend is

  1. keep the original expm1 name (that seems the most useful to people who are trying to get spark compatible behavior)
  2. Change to use a function to register spark compatible functions (see above)
  3. Change our sqlloigictest driver so it registers spark functions for any test that starts with spark_*.slt (similiar to pg_...)

That way most sqllogictest stuff stays the same, and we can write spark tests in spark/spark_math.slt, spark/spark_string.slt etc type tests

Here is the code that customizes the context for the individual test files

match file_name {
"information_schema_table_types.slt" => {
info!("Registering local temporary table");
register_temp_table(test_ctx.session_ctx()).await;
}

Comment on lines 24 to 26
SELECT spark_expm1(1::INT);
----
1.718281828459
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried running this in Spark 3.5.3 and did not get the same answer.

scala> spark.sql("select expm1(1)").show()
+-----------------+
|         EXPM1(1)|
+-----------------+
|1.718281828459045|
+-----------------+

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@andygrove I believe sqllogic is truncating the answer.

I initially had this test, which tested for the value 1.7182818284590453 (slightly more precise than your result) but removed it because cargo test (amd64) was giving the value 1.718281828459045 (https://github.com/apache/datafusion/actions/runs/13825914914/job/38680868216) while the rest of the cargo tests on different architectures were passing.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does sqllogictest have a way to test floating point results within some tolerance?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Doesn't look like it:

Remember, the purpose of sqllogictest is to validate the logic behind the evaluation of SQL statements, not the ability to handle extreme values. So keep content in a reasonable range: small integers, short strings, and floating point numbers that use only the most significant bits of an a 32-bit IEEE float.

https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't have to stick with sqllogictest.

We can create test helpers similar to:
https://github.com/lakehq/datafusion/blob/d78877a55c5e835a07a7ebf23a7bd515faf7d827/datafusion/optimizer/src/analyzer/type_coercion.rs#L2137-L2208

The above link is from an old PR that didn't end up getting merged in, but the general idea seems useful here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lol

query R
SELECT spark_expm1(1::INT);
----
1.718281828459

query T
SELECT spark_expm1(1::INT)::STRING;
----
1.7182818284590453

1.718281828459

query R
SELECT spark_expm1(a) FROM (VALUES (0::INT), (1::INT)) AS t(a);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd suggest adding tests for a wider range of values and edge cases, such as negative numbers, large positive and negative numbers, NaN, null, and so on.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call, will do!

50

query I
SELECT spark_ascii(a) FROM (VALUES ('Spark'), ('PySpark'), ('Pandas API')) AS t(a);
Copy link
Member

@andygrove andygrove Mar 14, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you add tests for edge cases?

Some ideas from ChatGPT (the results are from me actually running them in Spark):

scala> spark.sql("select ascii('😀')").show()
+---------+                                                                     
|ascii(😀)|
+---------+
|   128512|
+---------+

scala> spark.sql("select ascii('\n')").show()
+---------+
|ascii(\n)|
+---------+
|       10|
+---------+


scala> spark.sql("select ascii('\t')").show()
+---------+
|ascii(\t)|
+---------+
|        9|
+---------+

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call, will do!

@github-actions github-actions bot removed the core Core DataFusion crate label Apr 25, 2025
@shehabgamin shehabgamin requested a review from alamb April 26, 2025 08:58
@@ -193,7 +192,7 @@ macro_rules! get_row_value {
///
/// Floating numbers are rounded to have a consistent representation with the Postgres runner.
///
pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result<String> {
pub fn cell_to_string(col: &ArrayRef, row: usize, is_spark_path: bool) -> Result<String> {
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@alamb While digging into your suggestion (#15168 (comment)), I realized that we don't need to write an entire engine for Spark. All we care about is the logic in cell_to_string. For now, I haven’t created a Spark-specific spark_cell_to_string, since the issues we originally encountered with sqllogictest were related only to Float64 precision. We can always create a Spark-specificspark_cell_to_string later if we find that other changes are needed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice -- this makes sense to me -- I agree what you have here looks good

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @shehabgamin -- I think this looks great to me. I left a few comments but nothing that would block merging this PR and I think we can do them as follow on items.

Things that I think we should do next:

  1. Add an example somewhere (perhaps in the examples_directory) showing how to configure and use the spark functions in a SessionContext. I can help with this
  2. Automatically generate documentation for these functions, the way we do for other functions -- https://datafusion.apache.org/user-guide/sql/scalar_functions.html
  3. Test integrating this code into comet (with a draft PR or something) to make sure it works.

After this PR s merged, I suggest we implement one or two more small functions to give some example patterns to follow, and then I think we'll be ready to write a bunch of tickets to port all the functions

Screenshot 2025-04-28 at 8 43 03 PM

Cargo.lock Outdated
Comment on lines 2572 to 2577
"datafusion-functions-aggregate",
"datafusion-functions-aggregate-common",
"datafusion-functions-nested",
"datafusion-functions-table",
"datafusion-functions-window",
"datafusion-functions-window-common",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think these dependencies are used

Suggested change
"datafusion-functions-aggregate",
"datafusion-functions-aggregate-common",
"datafusion-functions-nested",
"datafusion-functions-table",
"datafusion-functions-window",
"datafusion-functions-window-common",

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice catch, done!

use std::sync::Arc;

/// Fluent-style API for creating `Expr`s
#[allow(unused)]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why does it need to be "allow unused"? I don't think this should be necessary for pub APIs

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Linter yells at me otherwise

/// Fluent-style API for creating `Expr`s
#[allow(unused)]
pub mod expr_fn {
pub use super::function::aggregate::expr_fn::*;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this list of modules ones that spark offers? I am not familiar with spark so I don't know off the top of my head

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, exactly!

@@ -1,5 +1,5 @@
Apache DataFusion
Copyright 2019-2024 The Apache Software Foundation
Copyright 2019-2025 The Apache Software Foundation
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

use std::sync::Arc;

#[user_doc(
doc_section(label = "Spark Math Functions"),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed per your suggestion here:
#15168 (comment)

@@ -193,7 +192,7 @@ macro_rules! get_row_value {
///
/// Floating numbers are rounded to have a consistent representation with the Postgres runner.
///
pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result<String> {
pub fn cell_to_string(col: &ArrayRef, row: usize, is_spark_path: bool) -> Result<String> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice -- this makes sense to me -- I agree what you have here looks good

}
}

pub(crate) fn spark_f64_to_str(value: f64) -> String {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this looks like a copy/paste of f64_to_str -- maybe we could just thread the spark flag down and avoid some duplication. Not necesary just a suggestion

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

f64_to_str is used in more than 1 place, so I figured it made sense to create a new function.

};
}

#[test]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My personal preference is to test them all from .slt rather than have any rust based tests unless there is something that can not be tested from .slt

For the different string types, we could perhaps cover the different string types using the same pattern as normal string tests -- see https://github.com/apache/datafusion/blob/main/datafusion/sqllogictest/test_files/string/README.md

However, I don't think this is required

@alamb
Copy link
Contributor

alamb commented Apr 30, 2025

This looks great to me -- I plan to merge it tomorrow and start collecting next steps in a new EPIC ticket unless someone beats me to it

@alamb
Copy link
Contributor

alamb commented May 1, 2025

I have filed an epic to track filling out the datafusion-spark crate:

I will file some subtickets for follow on work as well (e.g. what is in #15168 (review))

@alamb
Copy link
Contributor

alamb commented May 1, 2025

Onward!

@xudong963
Copy link
Member

Fyi, the main CI has failed since the PR

@blaginin blaginin mentioned this pull request May 2, 2025
@alamb
Copy link
Contributor

alamb commented May 2, 2025

Fyi, the main CI has failed since the PR

@blaginin has fixed it -- it appears to have been a logical conflict

@linhr linhr deleted the add-spark-crate branch May 14, 2025 03:10
# for free to join this conversation on GitHub. Already have an account? # to comment
Labels
functions Changes to functions implementation sqllogictest SQL Logic Tests (.slt)
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[DISCUSSION] Add separate crate to cover spark builtin functions
4 participants