Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

[FEAT] [SQL] Add global agg support for SQL #2799

Merged
merged 1 commit into from
Sep 9, 2024

Conversation

amitschang
Copy link
Contributor

This adds support to SQL to run global aggregation when there are only aggregations and no group by, which is inline with many other SQL implementations.

It is similar in nature to #1979, but specific to SQL - depending on how that issue is resolved the SQL planner code here could be redundant.

Example:

d = daft.from_pydict({"i": [1,2,3,4,5]})
daft.sql('''SELECT sum(i) FROM d''', SQLCatalog({'d': d})).collect()
╭───────╮                                                                                                    
│ i     │
│ ---   │
│ Int64 │
╞═══════╡
│ 15    │
╰───────╯

note: I know the SQL support is new, so sorry if this is jumping the gun! Is it better to first create discussions or issues for this kind of change? I've been looking forward to SQL in this project and from playing around with using sqlglot on the python side to build a logical plan, I noticed a few things - now that this is here I thought I might contribute 😄

@github-actions github-actions bot added the enhancement New feature or request label Sep 6, 2024
@@ -316,4 +317,17 @@ mod tests {

Ok(())
}

#[rstest]
fn test_global_agg(mut planner: SQLPlanner, tbl_1: LogicalPlanRef) -> SQLPlannerResult<()> {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm not totally sure this gets run in make test (and running via rust-analyzer didn't work out-of-the-box)

Copy link

codspeed-hq bot commented Sep 6, 2024

CodSpeed Performance Report

Merging #2799 will degrade performances by 18.98%

Comparing amitschang:sql-global-agg (d2230f9) with main (91d9fe9)

Summary

⚡ 2 improvements
❌ 1 regressions
✅ 13 untouched benchmarks

⚠️ Please fix the performance issues or acknowledge them on CodSpeed.

Benchmarks breakdown

Benchmark main amitschang:sql-global-agg Change
test_count[1 Small File] 19.6 ms 24.2 ms -18.98%
test_explain[100 Small Files] 49 ms 37.6 ms +30.14%
test_show[100 Small Files] 307.8 ms 49.7 ms ×6.2

@djouallah
Copy link
Contributor

djouallah commented Sep 8, 2024

I would really appreciate a syntax like this just for compatibility reason with duckdb/datafusion/polars/chdb

@jaychia let's just make life easy for new users

something like this

con = SQLCatalog({"test": daft.from_pydict({"a": [1, 2, 3]})})
con .sql("SELECT * FROM test").show()

@universalmind303 universalmind303 merged commit f9b7af2 into Eventual-Inc:main Sep 9, 2024
32 of 33 checks passed
@universalmind303
Copy link
Collaborator

universalmind303 commented Sep 9, 2024

I would really appreciate a syntax like this just for compatibility reason with duckdb/datafusion/polars/chdb

@jaychia let's just make life easy for new users

something like this

con = SQLCatalog({"test": daft.from_pydict({"a": [1, 2, 3]})})
con .sql("SELECT * FROM test").show()

@djouallah we do have an open issue for selecting from python objects (similar to duckdb and polars) that is pretty high on our priority list!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement New feature or request
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants