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
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 14 additions & 0 deletions src/daft-sql/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,7 @@ mod tests {
#[case::orderby("select * from tbl1 order by i32 asc")]
#[case::orderby_multi("select * from tbl1 order by i32 desc, f32 asc")]
#[case::whenthen("select case when i32 = 1 then 'a' else 'b' end from tbl1")]
#[case::globalagg("select max(i32) from tbl1")]
fn test_compiles(mut planner: SQLPlanner, #[case] query: &str) -> SQLPlannerResult<()> {
let plan = planner.plan_sql(query);
assert!(plan.is_ok(), "query: {}\nerror: {:?}", query, plan);
Expand Down Expand Up @@ -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)

let sql = "select max(i32) from tbl1";
let plan = planner.plan_sql(sql)?;

let expected = LogicalPlanBuilder::new(tbl_1, None)
.aggregate(vec![col("i32").max()], vec![])?
.build();

assert_eq!(plan, expected);
Ok(())
}
}
9 changes: 7 additions & 2 deletions src/daft-sql/src/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ use daft_dsl::{
numeric::{ceil, floor},
utf8::{ilike, like},
},
lit, literals_to_series, null_lit, Expr, ExprRef, LiteralValue, Operator,
has_agg, lit, literals_to_series, null_lit, Expr, ExprRef, LiteralValue, Operator,
};
use daft_plan::{LogicalPlanBuilder, LogicalPlanRef};

Expand Down Expand Up @@ -253,7 +253,12 @@ impl SQLPlanner {
rel.inner = rel.inner.aggregate(to_select, groupby_exprs)?;
} else if !to_select.is_empty() {
let rel = self.relation_mut();
rel.inner = rel.inner.select(to_select)?;
let has_aggs = to_select.iter().any(has_agg);
if has_aggs {
rel.inner = rel.inner.aggregate(to_select, vec![])?;
} else {
rel.inner = rel.inner.select(to_select)?;
}
}

Ok(())
Expand Down
17 changes: 17 additions & 0 deletions tests/sql/test_sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -96,3 +96,20 @@ def test_fizzbuzz_sql():
def test_sql_expr(actual, expected):
actual = daft.sql_expr(actual)
assert repr(actual) == repr(expected)


def test_sql_global_agg():
df = daft.from_pydict({"n": [1, 2, 3]})
catalog = SQLCatalog({"test": df})
df = daft.sql("SELECT max(n) max_n, sum(n) sum_n FROM test", catalog=catalog)
assert df.collect().to_pydict() == {"max_n": [3], "sum_n": [6]}
# If there is agg and non-agg, it should fail
with pytest.raises(Exception, match="Expected aggregation"):
daft.sql("SELECT n,max(n) max_n FROM test", catalog=catalog)


def test_sql_groupby_agg():
df = daft.from_pydict({"n": [1, 1, 2, 2], "v": [1, 2, 3, 4]})
catalog = SQLCatalog({"test": df})
df = daft.sql("SELECT sum(v) FROM test GROUP BY n ORDER BY n", catalog=catalog)
assert df.collect().to_pydict() == {"n": [1, 2], "v": [3, 7]}
Loading